zhipeng93 commented on a change in pull request #32:
URL: https://github.com/apache/flink-ml/pull/32#discussion_r756556452



##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayesModel.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.classification.naivebayes;
+
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.connector.file.src.FileSource;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.ml.api.core.Model;
+import org.apache.flink.ml.common.broadcast.BroadcastUtils;
+import org.apache.flink.ml.common.datastream.TableUtils;
+import org.apache.flink.ml.linalg.BLAS;
+import org.apache.flink.ml.linalg.Vector;
+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.streaming.api.functions.sink.filesystem.bucketassigners.BasePathBucketAssigner;
+import 
org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.OnCheckpointRollingPolicy;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+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.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.function.Function;
+
+/** A Model which classifies data using the model data computed by {@link 
NaiveBayes}. */
+public class NaiveBayesModel
+        implements Model<NaiveBayesModel>, 
NaiveBayesModelParams<NaiveBayesModel> {
+    private static final long serialVersionUID = -4673084154965905629L;
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+    private Table modelTable;
+
+    public NaiveBayesModel() {
+        ParamUtils.initializeMapWithDefaultValues(paramMap, this);
+    }
+
+    @Override
+    public Table[] transform(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+
+        final String predictionCol = getPredictionCol();
+        final String featuresCol = getFeaturesCol();
+        final String broadcastModelKey = "NaiveBayesModelStream";
+
+        RowTypeInfo inputTypeInfo = 
TableUtils.getRowTypeInfo(inputs[0].getResolvedSchema());
+        RowTypeInfo outputTypeInfo =
+                new RowTypeInfo(
+                        ArrayUtils.addAll(
+                                inputTypeInfo.getFieldTypes(), 
TypeInformation.of(Object.class)),
+                        ArrayUtils.addAll(inputTypeInfo.getFieldNames(), 
predictionCol));
+
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
modelTable).getTableEnvironment();
+        DataStream<NaiveBayesModelData> modelStream =
+                NaiveBayesModelData.toDataStream(tEnv, modelTable);
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        Map<String, DataStream<?>> broadcastMap = new HashMap<>();
+        broadcastMap.put(broadcastModelKey, modelStream);
+
+        Function<List<DataStream<?>>, DataStream<Row>> function =
+                dataStreams -> {
+                    DataStream stream = dataStreams.get(0);
+                    return stream.transform(
+                            this.getClass().getSimpleName(),
+                            outputTypeInfo,
+                            new PredictLabelOperator(
+                                    new PredictLabelFunction(featuresCol, 
broadcastModelKey)));
+                };
+        DataStream<Row> output =
+                BroadcastUtils.withBroadcastStream(
+                        Collections.singletonList(input), broadcastMap, 
function);
+
+        Table outputTable = tEnv.fromDataStream(output);
+
+        return new Table[] {outputTable};
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        StreamTableEnvironment tEnv =
+                (StreamTableEnvironment) ((TableImpl) 
modelTable).getTableEnvironment();
+
+        String dataPath = ReadWriteUtils.getDataPath(path);
+        FileSink<NaiveBayesModelData> sink =
+                FileSink.forRowFormat(
+                                new Path(dataPath), new 
NaiveBayesModelData.ModelDataEncoder())
+                        .withRollingPolicy(OnCheckpointRollingPolicy.build())
+                        .withBucketAssigner(new BasePathBucketAssigner<>())
+                        .build();
+        NaiveBayesModelData.toDataStream(tEnv, modelTable).sinkTo(sink);
+
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayesModel load(StreamExecutionEnvironment env, String 
path)
+            throws IOException {
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        Source<NaiveBayesModelData, ?, ?> source =
+                FileSource.forRecordStreamFormat(
+                                new 
NaiveBayesModelData.ModelDataStreamFormat(),
+                                ReadWriteUtils.getDataPaths(path))
+                        .build();
+        NaiveBayesModel model = ReadWriteUtils.loadStageParam(path);
+        DataStream<NaiveBayesModelData> modelData =
+                env.fromSource(source, WatermarkStrategy.noWatermarks(), 
"modelData");
+        model.setModelData(NaiveBayesModelData.fromDataStream(tEnv, 
modelData));
+
+        return model;
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    @Override
+    public NaiveBayesModel setModelData(Table... inputs) {
+        Preconditions.checkArgument(inputs.length == 1);
+        modelTable = inputs[0];
+        return this;
+    }
+
+    @Override
+    public Table[] getModelData() {
+        return new Table[] {modelTable};
+    }
+
+    private static class PredictLabelOperator
+            extends AbstractUdfStreamOperator<Row, PredictLabelFunction>
+            implements OneInputStreamOperator<Row, Row> {
+        public PredictLabelOperator(PredictLabelFunction userFunction) {
+            super(userFunction);
+        }
+
+        @Override
+        public void processElement(StreamRecord<Row> streamRecord) {

Review comment:
       > 
   
   Ah, I see. 
   Yes, the broadcast variables can only be accessed by 
`RichFunction#getRuntimeContext#getBroadcastVariable` now. 
   
   A possible solution to put processing logic in 
`PredictLabelOperator::processElement` is to add a AbstractFunction but only 
use it for accessing broadcast variables. An examples can be found in 
`org.apache.flink.ml.common.broadcast.operator.TestOneInputOp`
   
   




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to