Discussion threadhttps://lists.apache.org/thread/zgbyp5hjhsp2bs3t1txq2p1l5t3c08yt
Vote threadhttps://lists.apache.org/thread/pq1ot6wj1j87jxm4tqydl4vf6klqsy4l
JIRA

Unable to render Jira issues macro, execution error.

Releaseml-2.0.0

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

[This FLIP proposal is a joint work between Dong Lin and Zhipeng Zhang]

Motivation

An important part of the Flink ML library infrastructure is the APIs to define, set and get parameters of machine learning stages (e.g. Transformer, Estimator, AlgoOperator). Currently Flink ML library provides these APIs through the public methods of WithParams (an interface), ParamInfo, ParamInfoFactory and Params (three classes). In this FLIP we propose to simplify the Flink ML library infrastructure by reducing the number of classes as well as the number of public methods on those classes, and still support all the expected use-cases.

The goal of this FLIP is to increase developer velocity by making Flink ML library easier to use.

Issues with the existing parameter-related interface and classes

In this section we first summarize the public methods of the existing parameter-related interface and classes, followed by explanation of why they could be simplified.

1) The ParamInfo class provides the definition of a parameter, including its name, type, default value and so on.

public class ParamInfo<V> {
    private final String name;
    private final String[] alias;
    private final String description;
    private final boolean isOptional;
    private final boolean hasDefaultValue;
    private final V defaultValue;
    private final ParamValidator<V> validator;
    private final Class<V> valueClass;

    ParamInfo(String name, String[] alias, String description, boolean isOptional, boolean hasDefaultValue, V defaultValue, ParamValidator<V> validator, Class<V> valueClass) {...}

    public String getName() {...}

    public String[] getAlias() {...}

    public String getDescription() {...}

    public boolean isOptional() {...}

    public boolean hasDefaultValue() {...}

    public V getDefaultValue() {...}

    public ParamValidator<V> getValidator() {...}

    public Class<V> getValueClass() {...}
}


2) The WithParam interface provides APIs to set and get parameter values.

public interface WithParams<T> {

    Params getParams();

    default <V> T set(ParamInfo<V> info, V value) {
        getParams().set(info, value);
        return (T) this;
    }

    default <V> V get(ParamInfo<V> info) {
        return getParams().get(info);
    }
}


3) The ParamInfoFactory class provides APIs to build a ParamInfo.

public class ParamInfoFactory {

    public static <V> ParamInfoBuilder<V> createParamInfo(String name, Class<V> valueClass) {...}

    public static class ParamInfoBuilder<V> {
        ParamInfoBuilder(String name, Class<V> valueClass) {...}

        public ParamInfoBuilder<V> setAlias(String[] alias) {...}

        public ParamInfoBuilder<V> setDescription(String description) {...}

        public ParamInfoBuilder<V> setOptional() {...}

        public ParamInfoBuilder<V> setRequired() {...}

        public ParamInfoBuilder<V> setHasDefaultValue(V defaultValue) {...}

        public ParamInfoBuilder<V> setValidator(ParamValidator<V> validator) {...}

        public ParamInfo<V> build() {...}
    }
}


4) The Params class stores the mapping from parameter to value. And it provides APIs to convert ParamInfo to/from json-formatted string.

public class Params implements Serializable, Cloneable {
    public int size() {...}

    public void clear() {...}

    public boolean isEmpty() {...}

    public <V> V get(ParamInfo<V> info) {...}

    public <V> Params set(ParamInfo<V> info, V value) {...}

    public <V> void remove(ParamInfo<V> info) {...}

    public <V> boolean contains(ParamInfo<V> info) {...}

    public String toJson() {...}

    public void loadJson(String json) {...}

    public static Params fromJson(String json) {...}

    public Params merge(Params otherParams) {...}

    public Params clone() {...}
}


Here are the redundancy issues with the existing APIs:

1) ParamInfo does not need to the "String[] alias" field.

Alias is typically needed only when we need to migrate parameter of an existing machine learning algorithm to use a new name. Since there is no machine learning algorithm in the Flink ML library yet, we do not have any use-case for this field.

Many other frameworks/libraries, such as Apache Spark ML and Apache Kafka, can support their users without having alias for its configs. It would be better to follow this pattern of avoiding changing parameter names, instead of supporting alias from the very beginning.

2) ParameterInfo does not need the "boolean hasDefaultValue" field and the "boolean isOptional" field.

As far as we can tell, there is no use-case that requires these two fields.

The parameter definition itself does not specify how its value should be used. It is up to the algorithm to decide how to use parameter value and whether the parameter value is optional. Thus the algorithm, not the parameter definition, should decide whether the parameter is optional.

"hasDefaultValue" appears to be redundant because this information can be derived by checking whether the defaultValue == null.

3) ParamInfo does not need to have those getter methods. All its member fields could be declared as "public final" since there is no need to change those fields are ParamInfo is constructed.

4) ParamInfoFactory is un-necessary and we can just construct ParamInfo by calling the ParamInfo constructors.

5) Params is unnecessary after we remove the alias field from ParamInfo.

After we remove the alias field from ParamInfo, we can just replace Params with Map<ParamInfo<?>, Object>, which effectively contains the mapping from parameter definitions to parameter values.

Here are the usability issues with the existing APIs:

1) Params::loadJson() and Params::toJson(), which are used to save/load a stage, can not guarantee the same parameter values will be used, if the default value defined in the ParamInfo changes.

This behavior makes it hard to guarantee consistent accuracy/performance of an existing Transformer/Estimator.

2) The existing Params::get() and Params::set() implementations always convert the value from/to json-formatted string, which could incur unnecessary performance overhead.

It is possible that an algorithm will want to set and get parameters one or more times before saving the model to disk. Ideally this should not involve value serialization and deserialization overhead.

Public Interfaces

1) We propose to replace WithParams/ParamInfo/ParamInfoFactory/Params with Param and WithParams as shown below. And it could be shown that the proposed interface and classes address all the issues described above.

/**
 * Definition of a parameter, including name, class, description, default value and the validator.
 *
 * @param <T> The type of the parameter value
 */
public class Param<T> implements Serializable {
    public final String name;
    public final Class<T> clazz;
    public final String description;
    public final T defaultValue;
    public final ParamValidator<T> validator;

    public Param(String name, Class<T> clazz, String description, T defaultValue, ParamValidator<T> validator) {...}

    // Encodes the given object into a json-formatted string
    public String jsonEncode(T value) throws IOException {...}

    // Decodes the json-formatted string into an object of the given type.
    public T jsonDecode(String json) throws IOException {...}
}



/**
 * Interface for classes that take parameters. It provides APIs to set and get parameters.
 *
 * @param <T> The class type of WithParams implementation itself.
 */
@PublicEvolving
public interface WithParams<T> { 
    /**
     * Gets the parameter by its name.
     *
     * @param name The parameter name.
     * @param <V> The class type of the parameter value.
     * @return The parameter.
     */
    default <V> Param<V> getParam(String name) {...}

    /**
     * Sets the value of the parameter.
     *
     * @param param The parameter.
     * @param value The parameter value.
     * @return The WithParams instance itself.
     */
    @SuppressWarnings("unchecked")
    default <V> T set(Param<V> param, V value) {...}

    /**
     * Gets the value of the parameter.
     *
     * @param param The parameter.
     * @param <V> The class type of the parameter value.
     * @return The parameter value.
     */
    @SuppressWarnings("unchecked")
    default <V> V get(Param<V> param) {...}

    /**
     * Returns a map which should contain value for every parameter that meets one of the following
     * conditions.
     *
     * <p>1) set(...) has been called to set value for this parameter.
     *
     * <p>2) The parameter is a public final field of this WithParams instance. This includes fields
     * inherited from its interfaces and super-classes.
     *
     * <p>The subclass which implements this interface could meet this requirement by returning a
     * member field of the given map type, after having initialized this member field using the
     * {@link ParamUtils#initializeMapWithDefaultValues(Map, WithParams)} method.
     *
     * @return A map which maps parameter definition to parameter value.
     */
    Map<Param<?>, Object> getParamMap();
}


2) We propose to add the following subclasses of Param<?> to simplify the creation of parameters with primitive-typed values (e.g. long, int, boolean). 

public class BooleanParam extends Param<Boolean> {
  ...
}

public class IntParam extends Param<Integer> {
  ...
}

public class LongParam extends Param<Long> {
  ...
}

public class FloatParam extends Param<Float> {
  ...
}

public class DoubleParam extends Param<Double> {
  ...
}

public class StringParam extends Param<String> {
  ...
}

public class IntArrayParam extends Param<Integer[]> {
  ...
}

public class LongArrayParam extends Param<Long[]> {
  ...
}

public class FloatArrayParam extends Param<Float[]> {
  ...
}

public class DoubleArrayParam extends Param<Double[]> {
  ...
}


3) We propose to add the following util method to facilitate the initialization of parameter map with default parameter values.

/** Utility methods for reading and writing stages. */
public class ParamUtils {
    /**
     * Updates the paramMap with default values of all public final Param-typed fields of the given
     * instance. A parameter's value will not be updated if this parameter is already found in the
     * map.
     *
     * <p>Note: This method should be called after all public final Param-typed fields of the given
     * instance have been defined. A good choice is to call this method in the constructor of the
     * given instance.
     */
    public static void initializeMapWithDefaultValues(Map<Param<?>, Object> paramMap, WithParams<?> instance) {...}
}  


4) We propose to add the following subclasses of ParamValidator<?> to simplify the creation of parameter validators with numerical values.

/**
 * Factory methods for common validation functions. The numerical methods only support Int, Long,
 * Float, and Double.
 */
public class ParamValidators {
    // Always return true.
    public static <T> ParamValidator<T> alwaysTrue() {...}

    // Check if the parameter value is greater than lowerBound.
    public static <T> ParamValidator<T> gt(double lowerBound) {...}

    // Check if the parameter value is greater than or equal to lowerBound.
    public static <T> ParamValidator<T> gtEq(double lowerBound) {...}

    // Check if the parameter value is less than upperBound.
    public static <T> ParamValidator<T> lt(double upperBound) {...}

    // Check if the parameter value is less than or equal to upperBound.
    public static <T> ParamValidator<T> ltEq(double upperBound) {...}

    /**
     * Check if the parameter value is in the range from lowerBound to upperBound.
     *
     * @param lowerInclusive if true, range includes value = lowerBound
     * @param upperInclusive if true, range includes value = upperBound
     */
    public static <T> ParamValidator<T> inRange(double lowerBound, double upperBound, boolean lowerInclusive, boolean upperInclusive) {...}

    // Check if the parameter value is in the range [lowerBound, upperBound].
    public static <T> ParamValidator<T> inRange(double lowerBound, double upperBound) {...}

    // Check if the parameter value is in the array of allowed values.
    public static <T> ParamValidator<T> inArray(T... allowed) {...}

    // Check if the parameter value is not null.
    public static <T> ParamValidator<T> notNull() {...}
}

Proposed Changes

We make the following notes regarding the implementation and the usage of the proposed interfaces:

1) With the proposed interface, algorithm developers can define multiple parameters by calling e.g. Param<Boolean> BOOLEAN_PARAM = new BooleanParam(...). And in order to have WithParams::getParamMap() return those parameter values, regardless of whether algorithm users have explicitly called WithParams::set(...) to set parameter values, algorithm developer should make sure to initialize the paramMap in the constructor.

This can be achieved by calling the util method ParamUtils.initializeMapWithDefaultValues(paramMap, withParamsInstance).


2) The initializeMapWithDefaultValues(paramMap, withParamsInstance) method will use Java reflection to enumerate all public final fields of withParamInstance, find those fields assignable from the Param class, and update the given paramMap with default value for those Param fields that are not already found in the paramMap.

In order for this to work correctly, initializeMapWithDefaultValues(...) should be called after all public final Param-typed fields of the given WithParams instance have been defined. A good choice is to call this method in the constructor of the WithParams instance.


Example Usage

In the following we provide an example code snippet that shows how to define, set and get parameter values with various types.

More specifically, the code snippet covers the following feature:

1) How to define parameters of various primitive types (e.g. long, int array, string).

2) How to define a parameter as a static field in an interface and accesses this parameter.

3) How to define a  parameter as a non-static field in a class and accesses this parameter.

4) How to access a parameter by a Param<?> variable.

5) How to access a parameter by its name.


// An example interface that provides pre-defined parameters.
public interface MyParams<T> extends WithParams<T> {
    Param<Boolean> BOOLEAN_PARAM = new BooleanParam("booleanParam", "Description", false);

    Param<Integer> INT_PARAM = new IntParam("intParam", "Description", 1, ParamValidators.lt(100));

    Param<Long> LONG_PARAM = new LongParam("longParam", "Description", 2L, ParamValidators.lt(100));

    Param<Integer[]> INT_ARRAY_PARAM = new IntArrayParam("intArrayParam", "Description", new Integer[] {3, 4});

    Param<String[]> STRING_ARRAY_PARAM = new StringArrayParam("stringArrayParam", "Description", new String[] {"5", "6"});
}

// An example stage class that defines its own parameters and also inherits parameters from MyParams.
public static class MyStage implements Stage<MyStage>, MyParams<MyStage> {
    private final Map<Param<?>, Object> paramMap = new HashMap<>();

    public final Param<Integer> extraIntParam = new IntParam("extraIntParam", "Description", 100, ParamValidator.ALWAYS_TRUE);

    public MyStage() {
        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
    }

    // Skipped implementation of save() and load().
}


public static void main(String[] args) {
    MyStage stage = new MyStage();

    // Gets the value of a parameter defined in the MyParams interface without first setting its value.
    Long[] longArrayValue = stage.get(MyParams.LONG_ARRAY_PARAM);

    // Sets and gets value of a parameter defined in the MyParams interface.
    stage.set(MyParams.INT_PARAM, 1);
    Integer intValue = stage.get(MyParams.INT_PARAM);

    // Sets and gets value of a parameter defined in the MyStage class.
    stage.set(stage.extraIntParam, 2);
    Integer extraIntValue = stage.get(stage.extraIntParam);

    // Sets and gets value of a parameter identified by its name string.
    Param<?> longParam = stage.getParam("longParam");
    stage.set(longParam, 3L);
    Long longValue = (Long) stage.get(stage.getParam("longParam"));
}


Compatibility, Deprecation, and Migration Plan

The changes proposed in this FLIP is backward incompatible with the existing APIs of WithParams/ParamInfo/ParamInfoFactory/Params. We propose to change the APIs directly without deprecation period.

Since there is no implementation of Estimator/Transformer (excluding test-only implementations) in the existing Flink codebase, no work is needed to migrate the existing Flink codebase.

Test Plan

We will provide unit tests to validate the proposed changes.

Rejected Alternatives


1) Add the method "Map<Param<?>, Object> getInternalParmMap()" in in the WithParams interface. And do not add the initializeMapWithDefaultValues(...) util method.

In comparison to the proposed approach, this alternative approach makes the life a bit easier for the algorithm developer. And algorithm developer would not need to write code to invoke initializeMapWithDefaultValues(). The algorithm developer just needs to override getInternalParmMap() to return a member field of type Map<Param<?>, Object>. 

The downside of this proposed approach is that the algorithm users will see the getInternalParmMap() API in the WithParams interface that is never useful to them. The existence of getInternalParmMap() and getParamMap() on the same interface could be confusing to algorithm users.

We choose not to use this approach because we believe there will be much more algorithm users than algorithm developers. And it is more important to optimize the algorithm users' experience.