Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Status

Current state"Under Discussion"

...

JIRA: https://issues.apache.org/jira/browse/FLINK-22915

Released: 1.1415

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).


Table of Contents


NOTE: we have decided to replace this option with the design proposed here. This design doc will be deleted later.

Motivation and Use-cases

The existing Flink ML library allows users to compose an Estimator/Transformer from a pipeline (i.e. linear sequence) of Estimator/Transformer, and each Estimator/Transformer has one input and one output.

...

For example, some graph embedding algorithms (e.g., MetaPath2Vec) need to take two tables as inputs. These two tables represent nodes labels and edges of the graph respectively. This logic can be expressed as an Estimator with 2 input tables.

And some workflow may need to split 1 table into 2 tables, and use these tables for training and validation respectively. This logic can be expressed by a Transformer with 1 input table and 2 output tables.

2) Compose a directed-acyclic-graph Estimator/Transformer into an Estimator/Transformer.For example, the workflow may involve the join of 2 tables, where each table could be generated by a chain of Estimator/Transformer. The entire workflow is therefore a DAG of Estimator/Transformer.Express a generic machine learning function that has multiple inputs/outputs and make sure the class used for this purpose does not have any historical association with Estimator/Model concepts.

For example, some machine learning-related algorithms like model evaluation, computation of some statistics (e.g., ChiSquareTest), graph algorithms (e.g., PageRank) also take tables as input and produce tables as output. Though the functionality of Transformer API

can express this algorithm, some machine learning practitioners have the concern that Transformer is historically associated with the Estimator/Model/Feature Engineering concepts such that it feels weird to use Transformer for these use-cases.  

3) Online learning where a long-running instance Transformer needs to be updated by the latest model data generated by another long-running instance of Estimator.

In this scenario, we need to allow the Estimator to be run on a different machine than the Transformer. So that Estimator could consume sufficient computation resource in a cluster while the Transformer could be deployed on edge devices.

4) Provide APIs to allow Estimator/Transformer to be efficiently saved/loaded even if state (e.g. model data) of Estimator/Transformer is more than 10s of GBs.

The existing PipelineStage::toJson basically requires developer of Transformer/Estimator to serialize all model data into an in-memory string, which could be very inefficient (or practically impossible) if the model data is very large (e.g 10s of GBs).


In addition to addressing the above use-cases, this FLIP also proposes a few more changes to simplify the class hierarchy and improve API usability. The existing Flink ML library has the following usability issues:
4
5) The Model interface (which currently simply extends the Transformer interface without adding any extra logic) does not provide any added value (given that we already have Transformer). The added class hierarchy complexity is not justified.

56) fit/transform API requires users to explicitly provide the TableEnvironment, where the TableEnvironment could be retrieved from the Table instance given to the fit/transform.

67) A Pipeline is both a Transformer and an Estimator. The experience of using Pipeline is therefore different from the experience of using Estimator (with the needFit API).7) There is no API .

Background

Note: Readers who are familiar with the existing Estimator/Transformer/Pipeline APIs can skip this section.

The design proposed in this doc is built on top of the design proposed in FLIP-39: Flink ML pipeline and ML libs, which in turn is motivated by the Estimator/Transformer/Pipeline design proposed in Scikit-learn. Please feel free to read FLIP-39 and Scikit-learn paper for more detail. In this section, we explain the key background information to help better understand the motivation and benefits of the proposed design, as well as the definition of some terminology.

1) What is the motivation of defining the Estimator and the Transformer interfaces? How are these interfaces related to machine learning?

We expect most classic machine learning algorithms to have a training logic and inference logic. The training logic of the algorithm reads data (e.g. labeled data) and updates its variables. The inference logic of the algorithm makes prediction (e.g. label) for the unseen data.

The inference logic of this algorithm could be represented as a subclass of Transformer (say TransformerA). The Transformer subclass has a transform() method, which reads data (as Table) and outputs data (as Table).

The training logic of this algorithm could be represented as a subclass of Estimator (say EstimatorA). The Estimator subclass has a fit() method, which reads data (as Table) and outputs an instance of Transformer (e.g. TransformerA in this case). The TransformerA instance could be constructed using variables from the EstimatorA instance.

Then, in order to do training and prediction using this algorithm, a user could write the following code:

Code Block
languagejava
Estimator estimator = new EstimatorA(...)

Table prediction_result = estimator.fit(training_data).transform(inference_data)


2) How does Pipeline work? What is the motivation of the Pipeline class?

The Pipeline is created from a linear chain of Estimator/Transformer (referred to as stage below). And it is used as an Estimator (since it implements Estimator). 

The execution of Pipeline::fit is a bit more complicated than it looks, since the output of an Estimator stage is not used directly as the input of the next stage in the Pipeline.

We highlight the key execution detail of Pipeline::fit below:

  • The input of Pipeline::fit is given to the first stage in the Pipeline.
  • If a stage is Transformer, the given input is used to invoke Transformer::transform(...), whose output is used as the input of the next stage.
  • If a stage is Estimator, the given input is used to invoke Estimator::fit(...), which produces a Transformer instance. The same input is then given to the Transformer::transform of the fitted instance, whose output is used as the input of the next stage.
  • Eventually, any Estimator stage of this Pipeline will produce a Transformer, which are combined with the original Transformer stage of the Pipeline into a linear chain of Transformers. This linear chain of Transformers are composed into an instance of Transformer.

The motivation of the Pipeline class is to enable user to do the following:

  • Compose an Estimator from a linear chain of Estimator/Transformer
  • Use the Transformer instance fitted by this linear chain of Estimator/Transformer without additionally defining this Transformer instance (which should itself be a linear chain of Transformer) again.

It is important to make the following observation: if we don't provide the Pipeline class, users can still accomplish the same use-cases targeted by Pipeline by explicitly writing the training logic and inference logic separately using Estimator/Transformer APIs. But users would have to construct this chain of Estimator/Transformer twice (for training and inference respectively) provided by the Estimator/Transformer interface to validate the schema consistency of a Pipeline. Users would have to instantiate Tables (with I/O logics) and run fit/transform to know whether the stages in the Pipeline are compatible with each other.

Public Interfaces

This FLIP proposes quite a few changes and addition additions to the existing Flink ML APIs. We first describe the final APIs of the classes updated by this FLIP, followed by list of changes we have made to the Flink ML APIs.

Proposed Changes

...

the proposed API additions and changes, followed by the API code of interfaces and classes after making the proposed changes.

API additions and changes

Here we list the additions and the changes to the Flink ML API.

The following changes are the most important changes proposed by this doc:

1) Added the AlgoOperator class. AlgoOperator class has the same interface as the existing Transformer (i.e. has the transform method).

This change address the need to encode a generic multi-input multi-output machine learning function. 

2) Updated Transformer/Estimator to take list of tables as inputs and return list of tables as output.

This change addresses the use-cases described in the motivation section, e.g. a graph embedding Estimator needs to take 2 tables as inputs.

3) Added setStateStreams and getStateStreams to the Transformer interface.

This change addresses the use-cases described in the motivation section, where a running Transformer needs to ingest the model state streams emitted by a Estimator, which could be running on a different machine.

4) Removed the methods PipelineStage::toJson and PipelineStage::loadJson. Add methods save(...) and load(...) to the Stage interface.


The following changes are relatively minor:

5) Removed TableEnvironment from the parameter list of fit/transform APIs.

This change simplifies the usage of fit/transform APIs.

6) Added pipelineTransformer and let Pipeline implement only the Estimator. Pipeline is no longer a Transformer.

This change makes the experience of using Pipeline consistent with the experience of using Estimator/Transformer, where a class is either an Estimator or a Transformer.

7) Removed Pipeline::appendStage from the Pipeline class.

8) Removed the Model interface. And renamed PipelineModel to PipelineTransformer.

This change simplifies the class hierarchy by removing a redundant class. It follows the philosophy of only adding complexity when we have explicit use-case for it.

9) Renamed PipelineStage to Stage and add the PublicEvolving tag to the Stage interface.

Interfaces and classes after the proposed API changes

The following code block shows the interface of Stage, Transformer, Estimator, Pipeline and pipelineTransformer after making the changes listed above.

Code Block
languagejava
/**
 * Base class for a stage in a Pipeline. The interface is only a concept, and does not have any actual
 * functionality. Its subclasses could be Estimator, Transformer or AlgoOperator. No other classes should inherit this
 * interface directly.
 *
 * <p>Each stage is with parameters, and requires a public empty constructor for restoration.
 *
 * @param <T> The class type of the Stage implementation itself.
 * @see WithParams
 */
@PublicEvolving
interface Stage<T extends Stage<T>> extends WithParams<T>, Serializable {
    /**
     * Saves this stage to the given path.
     */
    void save(String path);

    /**
     * Loads this stage from the given path.
     */
    void load(String path);
}

/**
 * A AlgoOperator is a Stage that takes a list of tables as inputs and produces a list of
 * tables as results. It can be used to encode a generic multi-input multi-output machine learning function.
 *
 * @param <T> The class type of the AlgoOperator implementation itself.
 */
@PublicEvolving
public interface AlgoOperator<T extends AlgoOperator<T>> extends Stage<T> {
    /**
     * Applies the AlgoOperator on the given input tables, and returns the result tables.
     *
     * @param inputs a list of tables
     * @return a list of tables
     */
    Table[] transform(Table... inputs);
}

/**
 * A Transformer is a AlgoOperator with additional support for state streams, which could be set by the Estimator that fitted
 * this Transformer. Unlike AlgoOperator, a Transformer is typically associated with an Estimator.
 *
 * @param <T> The class type of the Transformer implementation itself.
 */
@PublicEvolving
public interface Transformer<T extends Transformer<T>> extends AlgoOperator<T> {
    /**
     * Uses the given list of tables to update internal states. This can be useful for e.g. online
     * learning where an Estimator fits an infinite stream of training samples and streams the model
     * diff data to this Transformer.
     *
     * <p>This method may be called at most once.
     *
     * @param inputs a list of tables
     */
    default void setStateStreams(Table... inputs) {
        throw new UnsupportedOperationException("this method is not implemented");
    }

    /**
     * Gets a list of tables representing changes of internal states of this Transformer. These
     * tables might come from the Estimator that instantiated this Transformer.
     *
     * @return a list of tables
     */
    default Table[] getStateStreams() {
        throw new UnsupportedOperationException("this method is not implemented");
    }
}

/**
 * An Estimator is a Stage that takes a list of tables as inputs and produces a Transformer.
 *
 * @param <E> class type of the Estimator implementation itself.
 * @param <M> class type of the Transformer this Estimator produces.
 */
@PublicEvolving
public interface Estimator<E extends Estimator<E, M>, M extends Transformer<M>> extends Stage<E> {
    /**
     * Trains on the given inputs and produces a Transformer.
     *
     * @param inputs a list of tables
     * @return a Transformer
     */
    M fit(Table... inputs);
}

/**
 * A Pipeline acts as an Estimator. It consists of an ordered list of stages, each of which could be
 * an Estimator, Transformer or AlgoOperator.
 */
@PublicEvolving
public final class Pipeline implements Estimator<Pipeline, pipelineTransformer> {

    public Pipeline(List<Stage<?>> stages) {...}

    @Override
    public pipelineTransformer fit(Table... inputs) {...}

    /** Skipped a few methods, including the implementations of the Estimator APIs. */
}

/**
 * A pipelineTransformer acts as a Transformer. It consists of an ordered list of Transformers or AlgoOperators.
 */
@PublicEvolving
public final class pipelineTransformer implements Transformer<pipelineTransformer> {

    public pipelineTransformer(List<Transformer<?>> transformers) {...}

    /** Skipped a few methods, including the implementations of the Transformer APIs. */
}


Example Usage

In this section we provide examples code snippets to demonstrate how we can use the APIs proposed in this FLIP to address the use-cases in the motivation section.

Online learning by running Transformer and Estimator concurrently on different machines

Here is an online learning scenario:

  • We have an infinite stream of tagged data that can be used for training.
  • We have an algorithm that can be trained using this infinite stream of data. This algorithm (with its latest states/parameters) can be used to do inference. And the accuracy of the algorithm increases with the increasing amount of training data it has seen.
  • We would like to train this algorithm using the given data stream on clusterA. And uses this algorithm with the update-to-date states/parameters to do inference on 10 different web servers.

In order to address this use-case, we can write the training and inference logics of this algorithm into an EstimatorA class and a TransformerA class with the following API behaviors:

  • EstimatorA::fit takes a table as input and returns an instance of TransformerA. Before this method returns this transformerA, it calls transformerA.setStateStreams(state_table), where the state_table represents the stream of algorithm parameters changes produced by EstimatorA.
  • TransformerA::setStateStreams(...) takes a table as input. Its implementation reads the data from this table to continuously update its algorithm parameters.
  • TransformerA::getStateStreams(...) returns the same table instance that has been provided via TransformerA::setStateStreams(...).
  • TransformerA::transform takes a table as input and returns a table. The returned table represents the inference results.


Here are the code snippets that address this use-case by using the proposed APIs.

First run the following code on clusterA:

Code Block
languagejava
void runTrainingOnClusterA(...) {
  // Creates the training stream from a Kafka topic.
  Table training_stream = ...;

  Estimator estimator = new EstimatorA(...);
  Transformer transformer = estimator.fit(training_stream);
  Table state_stream = transformer.getStateStreams()[0];

  // Writes the state_stream to a Kafka topicA.
  state_stream.sinkTable(...);
  // Saves transformer's state/metadata to a remote path.
  transformer.save(remote_path);

  // Executes the operators generated by the Estimator::fit(...), which reads from training_stream and writes to state_stream.
  env.execute()
}


Then run the following code on each web server:

Code Block
languagejava
void runInferenceOnWebServer(...) {
  // Creates the state stream from Kafka topicA which is written by the above code snippet. 
  Table state_stream = ...;
  // Creates the input stream that needs inference.
  Table input_stream = ...;

  Transformer transformer = new Transformer(...);
  transformer.load(remote_path);
  transformer.setStateStreams(new Table[]{state_stream});
  Table output_stream = transformer.transform(input_stream);

  // Do something with the output_stream.

  // Executes the operators generated by the Transformer::transform(...), which reads from state_stream to update its parameters. 
  // It also does inference on input_stream and produces results to the output_stream.
  env.execute()
}


Compatibility, Deprecation, and Migration Plan

The changes proposed in this FLIP is backward incompatible with the existing APIs. We propose to change the APIs directly without deprecation period. And we will manually migrate the existing open source projects which use the existing Flink ML API to use the proposed APIs.

Note that there is no implementation of Estimator/Transformer (excluding test-only implementations) in the existing Flink codebase. So the effort no work is needed to migrate the existing Flink codebase is zero.

To our best knowledge, the only open source project that uses the Flink ML API is https://github.com/alibaba/Alink. We will work together with Alink developers to migrate the existing code to use the proposed API. Furthermore, we will migrate Alink's Estimator/Transformer implementation to the Flink ML library codebase as much as possible.

Test Plan

We will provide unit tests to validate the proposed changes.

Rejected Alternatives

To be addedThere is no rejected alternatives to be listed here yet.