yunfengzhou-hub commented on a change in pull request #37: URL: https://github.com/apache/flink-ml/pull/37#discussion_r770241084
########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/onehotencoder/OneHotEncoder.java ########## @@ -0,0 +1,146 @@ +/* + * 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.ml.feature.onehotencoder; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.MapPartitionFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.ml.api.Estimator; +import org.apache.flink.ml.common.datastream.MapPartitionFunctionWrapper; +import org.apache.flink.ml.common.param.HasHandleInvalid; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; +import org.apache.flink.types.Row; +import org.apache.flink.util.Collector; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * An Estimator which implements the one-hot encoding algorithm. + * + * <p>See https://en.wikipedia.org/wiki/One-hot. + */ +public class OneHotEncoder + implements Estimator<OneHotEncoder, OneHotEncoderModel>, + OneHotEncoderParams<OneHotEncoder> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + + public OneHotEncoder() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + @Override + public OneHotEncoderModel fit(Table... inputs) { + Preconditions.checkArgument(inputs.length == 1); + Preconditions.checkArgument(getHandleInvalid().equals(HasHandleInvalid.ERROR_INVALID)); + + final String[] inputCols = getInputCols(); + + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) inputs[0]).getTableEnvironment(); + DataStream<Tuple2<Integer, Integer>> modelData = + tEnv.toDataStream(inputs[0]) + .flatMap(new ExtractInputColsValueFunction(inputCols)) + .keyBy(x -> x.f0) + .transform( + "findMaxIndex", + Types.TUPLE(Types.INT, Types.INT), + new MapPartitionFunctionWrapper<>(new FindMaxIndexFunction())); + + OneHotEncoderModel model = + new OneHotEncoderModel() + .setModelData(OneHotEncoderModelData.getModelDataTable(modelData)); + ReadWriteUtils.updateExistingParams(model, paramMap); + return model; + } + + @Override + public void save(String path) throws IOException { + ReadWriteUtils.saveMetadata(this, path); + } + + public static OneHotEncoder load(StreamExecutionEnvironment env, String path) + throws IOException { + return ReadWriteUtils.loadStageParam(path); + } + + @Override + public Map<Param<?>, Object> getParamMap() { + return paramMap; + } + + /** + * Extract values of input columns of input data. + * + * <p>Input: rows of input data containing designated input columns + * + * <p>Output: Pairs of column index and value stored in those columns + */ + private static class ExtractInputColsValueFunction + implements FlatMapFunction<Row, Tuple2<Integer, Integer>> { + private final String[] inputCols; + + private ExtractInputColsValueFunction(String[] inputCols) { + this.inputCols = inputCols; + } + + @Override + public void flatMap(Row row, Collector<Tuple2<Integer, Integer>> collector) { + for (int i = 0; i < inputCols.length; i++) { + Number number = (Number) row.getField(inputCols[i]); + Preconditions.checkArgument( + number.intValue() == number.doubleValue(), + String.format("Value %s cannot be parsed as indexed integer.", number)); + Preconditions.checkArgument( + number.intValue() >= 0, "Negative value not supported."); + collector.collect(new Tuple2<>(i, number.intValue())); + } + } + } + + /** Function to find the max index value for each column. */ + private static class FindMaxIndexFunction + implements MapPartitionFunction<Tuple2<Integer, Integer>, Tuple2<Integer, Integer>> { + + @Override + public void mapPartition( + Iterable<Tuple2<Integer, Integer>> iterable, + Collector<Tuple2<Integer, Integer>> collector) { + Map<Integer, Integer> map = new HashMap<>(); + for (Tuple2<Integer, Integer> value : iterable) { + map.put( + value.f0, + Math.max(map.getOrDefault(value.f0, Integer.MIN_VALUE), value.f1)); Review comment: As we discussed offline, this question is on the assumption that any categorical feature type is supported in one hot encoder. In fact the input integers should be indexed, which means if a large number appeared in input, then one hot encoder is supposed to create a sparse vector that long. With that said, I'll add more description in Java doc stating that only indexed integer is accepted as valid input. ########## File path: flink-ml-lib/src/main/java/org/apache/flink/ml/feature/onehotencoder/OneHotEncoderModel.java ########## @@ -0,0 +1,190 @@ +/* + * 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.ml.feature.onehotencoder; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.ml.api.Model; +import org.apache.flink.ml.common.broadcast.BroadcastUtils; +import org.apache.flink.ml.common.datastream.TableUtils; +import org.apache.flink.ml.common.param.HasHandleInvalid; +import org.apache.flink.ml.linalg.Vectors; +import org.apache.flink.ml.param.Param; +import org.apache.flink.ml.util.ParamUtils; +import org.apache.flink.ml.util.ReadWriteUtils; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableImpl; +import org.apache.flink.table.runtime.typeutils.ExternalTypeInfo; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +import org.apache.commons.lang3.ArrayUtils; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Vector; +import java.util.function.Function; + +/** + * A Model which encodes data into one-hot format using the model data computed by {@link + * OneHotEncoder}. + */ +public class OneHotEncoderModel + implements Model<OneHotEncoderModel>, OneHotEncoderParams<OneHotEncoderModel> { + private final Map<Param<?>, Object> paramMap = new HashMap<>(); + private Table modelDataTable; + + public OneHotEncoderModel() { + ParamUtils.initializeMapWithDefaultValues(paramMap, this); + } + + @Override + public Table[] transform(Table... inputs) { + final String[] inputCols = getInputCols(); + final String[] outputCols = getOutputCols(); + final boolean dropLast = getDropLast(); + final String broadcastModelKey = "OneHotModelStream"; + + Preconditions.checkArgument(getHandleInvalid().equals(HasHandleInvalid.ERROR_INVALID)); + Preconditions.checkArgument(inputs.length == 1); + Preconditions.checkArgument(inputCols.length == outputCols.length); + + RowTypeInfo inputTypeInfo = TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema()); + RowTypeInfo outputTypeInfo = + new RowTypeInfo( + ArrayUtils.addAll( + inputTypeInfo.getFieldTypes(), + Collections.nCopies( + outputCols.length, + ExternalTypeInfo.of(Vector.class)) + .toArray(new TypeInformation[0])), + ArrayUtils.addAll(inputTypeInfo.getFieldNames(), outputCols)); + + StreamTableEnvironment tEnv = + (StreamTableEnvironment) ((TableImpl) modelDataTable).getTableEnvironment(); + DataStream<Row> input = tEnv.toDataStream(inputs[0]); + DataStream<Tuple2<Integer, Integer>> modelStream = + OneHotEncoderModelData.getModelDataStream(modelDataTable); + + Map<String, DataStream<?>> broadcastMap = new HashMap<>(); Review comment: OK. I'll make the change. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
