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



##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayesParams.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.shared.colname.HasFeatureCols;
+import org.apache.flink.ml.param.shared.colname.HasLabelCol;
+import org.apache.flink.ml.param.shared.colname.HasPredictionCol;
+
+/**
+ * Parameters of naive bayes training process.
+ */
+public interface NaiveBayesParams<T> extends

Review comment:
       Could we add a parameter like modelType here? I suppose we will 
eventually expand NaiveBayes to support all modelTypes currently supported by 
Spark.

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayesParams.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.shared.colname.HasFeatureCols;
+import org.apache.flink.ml.param.shared.colname.HasLabelCol;
+import org.apache.flink.ml.param.shared.colname.HasPredictionCol;
+
+/**
+ * Parameters of naive bayes training process.
+ */
+public interface NaiveBayesParams<T> extends
+        HasFeatureCols<T>,
+        HasLabelCol<T>,
+        HasPredictionCol<T> {
+    Param<Double> SMOOTHING =
+            new DoubleParam(
+                    "smoothing",
+                    "the smoothing factor",
+                    0.0,

Review comment:
       The default value in Spark is 1.0. Do you think 1.0 would be a better 
default value here?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayesParams.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.ml.param.DoubleParam;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.param.ParamValidators;
+import org.apache.flink.ml.param.shared.colname.HasFeatureCols;
+import org.apache.flink.ml.param.shared.colname.HasLabelCol;
+import org.apache.flink.ml.param.shared.colname.HasPredictionCol;
+
+/**
+ * Parameters of naive bayes training process.
+ */
+public interface NaiveBayesParams<T> extends
+        HasFeatureCols<T>,
+        HasLabelCol<T>,
+        HasPredictionCol<T> {
+    Param<Double> SMOOTHING =
+            new DoubleParam(
+                    "smoothing",
+                    "the smoothing factor",

Review comment:
       How about we unanimously start the parameter doc with an upper-case 
character and end it with `.`, e.g. `The smoothing factor.`. Same for other 
parameters. I will update the KMeans PR to follow the same convention.
   
   Another minor question is, would `smoothing parameter` be better here? If I 
search `naive bayes smoothing` on Google, the next suggested word is 
`parameter` instead of `factor`. Spark also uses `parameter` in this case.

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");

Review comment:
       It seems a bit verbose to explicitly check whether input is null in 
every function. It is OK to just throw NPE in this case. Can we remove this 
check?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");
+
+        StreamTableEnvironment tEnv = (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        DataStream<NaiveBayesModelData> naiveBayesModel = input
+                .flatMap(new FlattenFunction(
+                        featureColNames,
+                        labelColName
+                ))
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple3<>(value.f0, value.f1, value.f2))
+                .window(EndOfStreamWindows.get())
+                .reduce((ReduceFunction<Tuple4<Object, Integer, Object, 
Double>>) (t0, t1) -> {t0.f3 += t1.f3; return t0; })
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple2<>(value.f0, value.f1))
+                .window(EndOfStreamWindows.get())
+                .aggregate(new ValueMapFunction())
+                .keyBy((KeySelector<Tuple4<Object, Integer, Map<Object, 
Double>, Double>, Object>) value -> value.f0)
+                .window(EndOfStreamWindows.get())
+                .aggregate(new MapArrayFunction(featureColNames.length))
+                .windowAll(EndOfStreamWindows.get())
+                .apply(new GenerateModelFunction(
+                        smoothing,
+                        featureColNames));
+
+        NaiveBayesModel model = new NaiveBayesModel()
+                .setPredictionCol(predictionCol)
+                .setFeatureCols(featureColNames);
+        model.setModelData(
+                tEnv.fromDataStream(naiveBayesModel)
+        );
+        return model;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayes load(String path) throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    /**
+     * Function to convert each column into tuples of label, feature column 
index, feature value, weight.
+     */
+    private static class FlattenFunction implements FlatMapFunction<Row, 
Tuple4<Object, Integer, Object, Double>> {
+        private final String[] featureColNames;
+        private final String labelColName;
+        private final int featureSize;
+
+        private FlattenFunction(String[] featureColNames, String labelColName) 
{
+            this.labelColName = labelColName;
+            this.featureColNames = featureColNames;
+            this.featureSize = featureColNames.length;
+        }
+
+        @Override
+        public void flatMap(Row row, Collector<Tuple4<Object, Integer, Object, 
Double>> collector) {
+            Object label = row.getField(labelColName);
+            if (label == null) {
+                return;
+            }
+
+            for (int i = 0; i < featureSize; i++) {
+                Object feature = row.getField(featureColNames[i]);
+                if (feature == null) {
+                    continue;
+                }
+                collector.collect(new Tuple4<>(label, i, feature, 1.0));
+            }
+        }
+    }
+
+    /**
+     * Function to aggregate feature value and weight into map from records 
with the same label and feature column index.
+     */
+    private static class ValueMapFunction implements AggregateFunction<
+            Tuple4<Object, Integer, Object, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>> {
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> 
createAccumulator() {
+            return new Tuple4<>(new Object(), -1, new HashMap<>(), 0.);
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> add(
+                Tuple4<Object, Integer, Object, Double> value,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            acc.f0 = value.f0;
+            acc.f1 = value.f1;
+            acc.f2.put(value.f2, value.f3);
+            acc.f3 += value.f3;
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> getResult(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> merge(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc0,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc1) {
+            if (acc0.f0.equals(new Object())) {
+                acc0.f0 = acc1.f0;
+            }
+            if (acc0.f1 == -1) {
+                acc0.f1 = acc1.f1;
+            }
+            for (Map.Entry<Object, Double> entry: acc1.f2.entrySet()) {
+                acc0.f2.put(entry.getKey(), entry.getValue());
+            }
+            acc0.f3 += acc1.f3;
+
+            return acc0;
+        }
+    }
+
+    /**
+     * Function to aggregate maps under the same label into arrays. array len 
= featureSize
+     */
+    private static class MapArrayFunction implements AggregateFunction<
+            Tuple4<Object, Integer, Map<Object, Double>, Double>,
+            Tuple3<Object, Double[], Map <Object, Double>[]>,
+            Tuple3<Object, Double[], Map <Object, Double>[]>> {
+        private final int featureSize;
+
+        private MapArrayFunction(int featureSize) {
+            this.featureSize = featureSize;
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> 
createAccumulator() {
+            Double[] weightSum = new Double[featureSize];
+            Arrays.fill(weightSum, 0.);
+            return new Tuple3<>(new Object(), weightSum, new 
HashMap[featureSize]);
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> add(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> value,
+                Tuple3<Object, Double[], Map<Object, Double>[]> acc) {
+            acc.f0 = value.f0;
+            acc.f1[value.f1] = value.f3;
+            acc.f2[value.f1] = value.f2;
+
+            return acc;
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> 
getResult(Tuple3<Object, Double[], Map<Object, Double>[]> acc) {
+            return acc;
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> merge(
+                Tuple3<Object, Double[], Map<Object, Double>[]> acc0,
+                Tuple3<Object, Double[], Map<Object, Double>[]> acc1) {
+            if (acc0.f0.equals(new Object())) {
+                acc0.f0 = acc1.f0;
+            }
+            for (int i = 0; i < featureSize; i++) {
+                acc0.f1[i] += acc1.f1[i];
+                if (acc1.f2[i] != null) {

Review comment:
       Would this ever happen?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");
+
+        StreamTableEnvironment tEnv = (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        DataStream<NaiveBayesModelData> naiveBayesModel = input
+                .flatMap(new FlattenFunction(
+                        featureColNames,
+                        labelColName
+                ))
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple3<>(value.f0, value.f1, value.f2))
+                .window(EndOfStreamWindows.get())
+                .reduce((ReduceFunction<Tuple4<Object, Integer, Object, 
Double>>) (t0, t1) -> {t0.f3 += t1.f3; return t0; })
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple2<>(value.f0, value.f1))
+                .window(EndOfStreamWindows.get())
+                .aggregate(new ValueMapFunction())
+                .keyBy((KeySelector<Tuple4<Object, Integer, Map<Object, 
Double>, Double>, Object>) value -> value.f0)
+                .window(EndOfStreamWindows.get())
+                .aggregate(new MapArrayFunction(featureColNames.length))
+                .windowAll(EndOfStreamWindows.get())
+                .apply(new GenerateModelFunction(
+                        smoothing,
+                        featureColNames));
+
+        NaiveBayesModel model = new NaiveBayesModel()
+                .setPredictionCol(predictionCol)
+                .setFeatureCols(featureColNames);
+        model.setModelData(
+                tEnv.fromDataStream(naiveBayesModel)
+        );
+        return model;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayes load(String path) throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    /**
+     * Function to convert each column into tuples of label, feature column 
index, feature value, weight.
+     */
+    private static class FlattenFunction implements FlatMapFunction<Row, 
Tuple4<Object, Integer, Object, Double>> {
+        private final String[] featureColNames;
+        private final String labelColName;
+        private final int featureSize;
+
+        private FlattenFunction(String[] featureColNames, String labelColName) 
{
+            this.labelColName = labelColName;
+            this.featureColNames = featureColNames;
+            this.featureSize = featureColNames.length;
+        }
+
+        @Override
+        public void flatMap(Row row, Collector<Tuple4<Object, Integer, Object, 
Double>> collector) {
+            Object label = row.getField(labelColName);
+            if (label == null) {
+                return;
+            }
+
+            for (int i = 0; i < featureSize; i++) {
+                Object feature = row.getField(featureColNames[i]);
+                if (feature == null) {
+                    continue;
+                }
+                collector.collect(new Tuple4<>(label, i, feature, 1.0));
+            }
+        }
+    }
+
+    /**
+     * Function to aggregate feature value and weight into map from records 
with the same label and feature column index.
+     */
+    private static class ValueMapFunction implements AggregateFunction<
+            Tuple4<Object, Integer, Object, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>> {
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> 
createAccumulator() {
+            return new Tuple4<>(new Object(), -1, new HashMap<>(), 0.);
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> add(
+                Tuple4<Object, Integer, Object, Double> value,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            acc.f0 = value.f0;
+            acc.f1 = value.f1;
+            acc.f2.put(value.f2, value.f3);
+            acc.f3 += value.f3;
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> getResult(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> merge(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc0,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc1) {
+            if (acc0.f0.equals(new Object())) {
+                acc0.f0 = acc1.f0;
+            }
+            if (acc0.f1 == -1) {
+                acc0.f1 = acc1.f1;
+            }
+            for (Map.Entry<Object, Double> entry: acc1.f2.entrySet()) {

Review comment:
       It seems simpler to do `acc0.f2.putAll(acc1.f2)`?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");
+
+        StreamTableEnvironment tEnv = (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        DataStream<NaiveBayesModelData> naiveBayesModel = input
+                .flatMap(new FlattenFunction(
+                        featureColNames,
+                        labelColName
+                ))
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple3<>(value.f0, value.f1, value.f2))
+                .window(EndOfStreamWindows.get())
+                .reduce((ReduceFunction<Tuple4<Object, Integer, Object, 
Double>>) (t0, t1) -> {t0.f3 += t1.f3; return t0; })
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple2<>(value.f0, value.f1))
+                .window(EndOfStreamWindows.get())
+                .aggregate(new ValueMapFunction())
+                .keyBy((KeySelector<Tuple4<Object, Integer, Map<Object, 
Double>, Double>, Object>) value -> value.f0)
+                .window(EndOfStreamWindows.get())
+                .aggregate(new MapArrayFunction(featureColNames.length))
+                .windowAll(EndOfStreamWindows.get())
+                .apply(new GenerateModelFunction(
+                        smoothing,
+                        featureColNames));
+
+        NaiveBayesModel model = new NaiveBayesModel()
+                .setPredictionCol(predictionCol)
+                .setFeatureCols(featureColNames);
+        model.setModelData(
+                tEnv.fromDataStream(naiveBayesModel)
+        );
+        return model;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayes load(String path) throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    /**
+     * Function to convert each column into tuples of label, feature column 
index, feature value, weight.
+     */
+    private static class FlattenFunction implements FlatMapFunction<Row, 
Tuple4<Object, Integer, Object, Double>> {
+        private final String[] featureColNames;
+        private final String labelColName;
+        private final int featureSize;
+
+        private FlattenFunction(String[] featureColNames, String labelColName) 
{
+            this.labelColName = labelColName;
+            this.featureColNames = featureColNames;
+            this.featureSize = featureColNames.length;
+        }
+
+        @Override
+        public void flatMap(Row row, Collector<Tuple4<Object, Integer, Object, 
Double>> collector) {
+            Object label = row.getField(labelColName);
+            if (label == null) {
+                return;
+            }
+
+            for (int i = 0; i < featureSize; i++) {
+                Object feature = row.getField(featureColNames[i]);
+                if (feature == null) {
+                    continue;
+                }
+                collector.collect(new Tuple4<>(label, i, feature, 1.0));
+            }
+        }
+    }
+
+    /**
+     * Function to aggregate feature value and weight into map from records 
with the same label and feature column index.
+     */
+    private static class ValueMapFunction implements AggregateFunction<
+            Tuple4<Object, Integer, Object, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>> {
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> 
createAccumulator() {
+            return new Tuple4<>(new Object(), -1, new HashMap<>(), 0.);

Review comment:
       It seems that the last field (i.e. weight) can be removed since this 
information can be derived from the 3rd field (i.e. the map from feature value 
to weight).

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");
+
+        StreamTableEnvironment tEnv = (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        DataStream<NaiveBayesModelData> naiveBayesModel = input
+                .flatMap(new FlattenFunction(
+                        featureColNames,
+                        labelColName
+                ))
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple3<>(value.f0, value.f1, value.f2))
+                .window(EndOfStreamWindows.get())
+                .reduce((ReduceFunction<Tuple4<Object, Integer, Object, 
Double>>) (t0, t1) -> {t0.f3 += t1.f3; return t0; })
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple2<>(value.f0, value.f1))
+                .window(EndOfStreamWindows.get())
+                .aggregate(new ValueMapFunction())
+                .keyBy((KeySelector<Tuple4<Object, Integer, Map<Object, 
Double>, Double>, Object>) value -> value.f0)
+                .window(EndOfStreamWindows.get())
+                .aggregate(new MapArrayFunction(featureColNames.length))
+                .windowAll(EndOfStreamWindows.get())
+                .apply(new GenerateModelFunction(
+                        smoothing,
+                        featureColNames));
+
+        NaiveBayesModel model = new NaiveBayesModel()
+                .setPredictionCol(predictionCol)
+                .setFeatureCols(featureColNames);
+        model.setModelData(
+                tEnv.fromDataStream(naiveBayesModel)
+        );
+        return model;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayes load(String path) throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    /**
+     * Function to convert each column into tuples of label, feature column 
index, feature value, weight.
+     */
+    private static class FlattenFunction implements FlatMapFunction<Row, 
Tuple4<Object, Integer, Object, Double>> {

Review comment:
       Supporting arbitrary object as label/feature seems a bit too wide. What 
is the practice followed by Spark?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");

Review comment:
       This could be checked by using `ParamValidators::notNull` for the 
labelColumn parameter?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");
+
+        StreamTableEnvironment tEnv = (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        DataStream<NaiveBayesModelData> naiveBayesModel = input
+                .flatMap(new FlattenFunction(
+                        featureColNames,
+                        labelColName
+                ))
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple3<>(value.f0, value.f1, value.f2))
+                .window(EndOfStreamWindows.get())
+                .reduce((ReduceFunction<Tuple4<Object, Integer, Object, 
Double>>) (t0, t1) -> {t0.f3 += t1.f3; return t0; })
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple2<>(value.f0, value.f1))
+                .window(EndOfStreamWindows.get())
+                .aggregate(new ValueMapFunction())
+                .keyBy((KeySelector<Tuple4<Object, Integer, Map<Object, 
Double>, Double>, Object>) value -> value.f0)
+                .window(EndOfStreamWindows.get())
+                .aggregate(new MapArrayFunction(featureColNames.length))
+                .windowAll(EndOfStreamWindows.get())
+                .apply(new GenerateModelFunction(
+                        smoothing,
+                        featureColNames));
+
+        NaiveBayesModel model = new NaiveBayesModel()
+                .setPredictionCol(predictionCol)
+                .setFeatureCols(featureColNames);
+        model.setModelData(
+                tEnv.fromDataStream(naiveBayesModel)
+        );
+        return model;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayes load(String path) throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    /**
+     * Function to convert each column into tuples of label, feature column 
index, feature value, weight.
+     */
+    private static class FlattenFunction implements FlatMapFunction<Row, 
Tuple4<Object, Integer, Object, Double>> {
+        private final String[] featureColNames;
+        private final String labelColName;
+        private final int featureSize;
+
+        private FlattenFunction(String[] featureColNames, String labelColName) 
{
+            this.labelColName = labelColName;
+            this.featureColNames = featureColNames;
+            this.featureSize = featureColNames.length;
+        }
+
+        @Override
+        public void flatMap(Row row, Collector<Tuple4<Object, Integer, Object, 
Double>> collector) {
+            Object label = row.getField(labelColName);
+            if (label == null) {
+                return;
+            }
+
+            for (int i = 0; i < featureSize; i++) {
+                Object feature = row.getField(featureColNames[i]);
+                if (feature == null) {
+                    continue;
+                }
+                collector.collect(new Tuple4<>(label, i, feature, 1.0));
+            }
+        }
+    }
+
+    /**
+     * Function to aggregate feature value and weight into map from records 
with the same label and feature column index.
+     */
+    private static class ValueMapFunction implements AggregateFunction<
+            Tuple4<Object, Integer, Object, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>> {
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> 
createAccumulator() {
+            return new Tuple4<>(new Object(), -1, new HashMap<>(), 0.);
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> add(
+                Tuple4<Object, Integer, Object, Double> value,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            acc.f0 = value.f0;
+            acc.f1 = value.f1;
+            acc.f2.put(value.f2, value.f3);
+            acc.f3 += value.f3;
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> getResult(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> merge(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc0,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc1) {
+            if (acc0.f0.equals(new Object())) {

Review comment:
       Would it be simpler to also do `acc0.f1 = acc1.f1` here?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");
+
+        StreamTableEnvironment tEnv = (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        DataStream<NaiveBayesModelData> naiveBayesModel = input
+                .flatMap(new FlattenFunction(
+                        featureColNames,
+                        labelColName
+                ))
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple3<>(value.f0, value.f1, value.f2))
+                .window(EndOfStreamWindows.get())
+                .reduce((ReduceFunction<Tuple4<Object, Integer, Object, 
Double>>) (t0, t1) -> {t0.f3 += t1.f3; return t0; })
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple2<>(value.f0, value.f1))
+                .window(EndOfStreamWindows.get())
+                .aggregate(new ValueMapFunction())
+                .keyBy((KeySelector<Tuple4<Object, Integer, Map<Object, 
Double>, Double>, Object>) value -> value.f0)
+                .window(EndOfStreamWindows.get())
+                .aggregate(new MapArrayFunction(featureColNames.length))
+                .windowAll(EndOfStreamWindows.get())
+                .apply(new GenerateModelFunction(
+                        smoothing,
+                        featureColNames));
+
+        NaiveBayesModel model = new NaiveBayesModel()
+                .setPredictionCol(predictionCol)
+                .setFeatureCols(featureColNames);
+        model.setModelData(
+                tEnv.fromDataStream(naiveBayesModel)
+        );
+        return model;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayes load(String path) throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    /**
+     * Function to convert each column into tuples of label, feature column 
index, feature value, weight.
+     */
+    private static class FlattenFunction implements FlatMapFunction<Row, 
Tuple4<Object, Integer, Object, Double>> {
+        private final String[] featureColNames;
+        private final String labelColName;
+        private final int featureSize;
+
+        private FlattenFunction(String[] featureColNames, String labelColName) 
{
+            this.labelColName = labelColName;
+            this.featureColNames = featureColNames;
+            this.featureSize = featureColNames.length;
+        }
+
+        @Override
+        public void flatMap(Row row, Collector<Tuple4<Object, Integer, Object, 
Double>> collector) {
+            Object label = row.getField(labelColName);
+            if (label == null) {

Review comment:
       Should we throw exception here if `label` is not found in the data?
   
   Sam for `feature`.

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,

Review comment:
       Why do we need to explicitly check this? If we do need this, could this 
be enforced as a validator in the parameter definition?

##########
File path: 
flink-ml-lib/src/main/java/org/apache/flink/ml/classification/naivebayes/NaiveBayes.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.functions.AggregateFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.ml.api.core.Estimator;
+import org.apache.flink.ml.common.datastream.EndOfStreamWindows;
+import org.apache.flink.ml.param.Param;
+import org.apache.flink.ml.util.ReadWriteUtils;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+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.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+
+/**
+ * Naive Bayes classifier is a simple probability classification algorithm 
using
+ * Bayes theorem based on independent assumption. It is an independent feature 
model.
+ * The input feature can be continual or categorical.
+ */
+public class NaiveBayes implements Estimator<NaiveBayes, NaiveBayesModel>,
+        NaiveBayesParams<NaiveBayes> {
+    private final Map<Param<?>, Object> paramMap = new HashMap<>();
+
+    @Override
+    public NaiveBayesModel fit(Table... inputs) {
+        String[] featureColNames = getFeatureCols();
+        String labelColName = getLabelCol();
+        String predictionCol = getPredictionCol();
+        double smoothing = getSmoothing();
+
+        Preconditions.checkNotNull(inputs, "input table list should not be 
null");
+        Preconditions.checkArgument(inputs.length == 1, "input table list 
should contain only one argument");
+        Preconditions.checkArgument(
+                new HashSet<>(Arrays.asList(featureColNames)).size() == 
featureColNames.length,
+                "feature columns should not duplicate");
+        Preconditions.checkNotNull(labelColName, "label column should be set");
+
+        StreamTableEnvironment tEnv = (StreamTableEnvironment) ((TableImpl) 
inputs[0]).getTableEnvironment();
+        DataStream<Row> input = tEnv.toDataStream(inputs[0]);
+
+        DataStream<NaiveBayesModelData> naiveBayesModel = input
+                .flatMap(new FlattenFunction(
+                        featureColNames,
+                        labelColName
+                ))
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple3<>(value.f0, value.f1, value.f2))
+                .window(EndOfStreamWindows.get())
+                .reduce((ReduceFunction<Tuple4<Object, Integer, Object, 
Double>>) (t0, t1) -> {t0.f3 += t1.f3; return t0; })
+                .keyBy((KeySelector<Tuple4<Object, Integer, Object, Double>, 
Object>) value -> new Tuple2<>(value.f0, value.f1))
+                .window(EndOfStreamWindows.get())
+                .aggregate(new ValueMapFunction())
+                .keyBy((KeySelector<Tuple4<Object, Integer, Map<Object, 
Double>, Double>, Object>) value -> value.f0)
+                .window(EndOfStreamWindows.get())
+                .aggregate(new MapArrayFunction(featureColNames.length))
+                .windowAll(EndOfStreamWindows.get())
+                .apply(new GenerateModelFunction(
+                        smoothing,
+                        featureColNames));
+
+        NaiveBayesModel model = new NaiveBayesModel()
+                .setPredictionCol(predictionCol)
+                .setFeatureCols(featureColNames);
+        model.setModelData(
+                tEnv.fromDataStream(naiveBayesModel)
+        );
+        return model;
+    }
+
+    @Override
+    public void save(String path) throws IOException {
+        ReadWriteUtils.saveMetadata(this, path);
+    }
+
+    public static NaiveBayes load(String path) throws IOException {
+        return ReadWriteUtils.loadStageParam(path);
+    }
+
+    @Override
+    public Map<Param<?>, Object> getParamMap() {
+        return paramMap;
+    }
+
+    /**
+     * Function to convert each column into tuples of label, feature column 
index, feature value, weight.
+     */
+    private static class FlattenFunction implements FlatMapFunction<Row, 
Tuple4<Object, Integer, Object, Double>> {
+        private final String[] featureColNames;
+        private final String labelColName;
+        private final int featureSize;
+
+        private FlattenFunction(String[] featureColNames, String labelColName) 
{
+            this.labelColName = labelColName;
+            this.featureColNames = featureColNames;
+            this.featureSize = featureColNames.length;
+        }
+
+        @Override
+        public void flatMap(Row row, Collector<Tuple4<Object, Integer, Object, 
Double>> collector) {
+            Object label = row.getField(labelColName);
+            if (label == null) {
+                return;
+            }
+
+            for (int i = 0; i < featureSize; i++) {
+                Object feature = row.getField(featureColNames[i]);
+                if (feature == null) {
+                    continue;
+                }
+                collector.collect(new Tuple4<>(label, i, feature, 1.0));
+            }
+        }
+    }
+
+    /**
+     * Function to aggregate feature value and weight into map from records 
with the same label and feature column index.
+     */
+    private static class ValueMapFunction implements AggregateFunction<
+            Tuple4<Object, Integer, Object, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>,
+            Tuple4<Object, Integer, Map<Object, Double>, Double>> {
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> 
createAccumulator() {
+            return new Tuple4<>(new Object(), -1, new HashMap<>(), 0.);
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> add(
+                Tuple4<Object, Integer, Object, Double> value,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            acc.f0 = value.f0;
+            acc.f1 = value.f1;
+            acc.f2.put(value.f2, value.f3);
+            acc.f3 += value.f3;
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> getResult(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc) {
+            return acc;
+        }
+
+        @Override
+        public Tuple4<Object, Integer, Map<Object, Double>, Double> merge(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc0,
+                Tuple4<Object, Integer, Map<Object, Double>, Double> acc1) {
+            if (acc0.f0.equals(new Object())) {
+                acc0.f0 = acc1.f0;
+            }
+            if (acc0.f1 == -1) {
+                acc0.f1 = acc1.f1;
+            }
+            for (Map.Entry<Object, Double> entry: acc1.f2.entrySet()) {
+                acc0.f2.put(entry.getKey(), entry.getValue());
+            }
+            acc0.f3 += acc1.f3;
+
+            return acc0;
+        }
+    }
+
+    /**
+     * Function to aggregate maps under the same label into arrays. array len 
= featureSize
+     */
+    private static class MapArrayFunction implements AggregateFunction<
+            Tuple4<Object, Integer, Map<Object, Double>, Double>,
+            Tuple3<Object, Double[], Map <Object, Double>[]>,
+            Tuple3<Object, Double[], Map <Object, Double>[]>> {
+        private final int featureSize;
+
+        private MapArrayFunction(int featureSize) {
+            this.featureSize = featureSize;
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> 
createAccumulator() {
+            Double[] weightSum = new Double[featureSize];
+            Arrays.fill(weightSum, 0.);
+            return new Tuple3<>(new Object(), weightSum, new 
HashMap[featureSize]);
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> add(
+                Tuple4<Object, Integer, Map<Object, Double>, Double> value,
+                Tuple3<Object, Double[], Map<Object, Double>[]> acc) {
+            acc.f0 = value.f0;
+            acc.f1[value.f1] = value.f3;
+            acc.f2[value.f1] = value.f2;
+
+            return acc;
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> 
getResult(Tuple3<Object, Double[], Map<Object, Double>[]> acc) {
+            return acc;
+        }
+
+        @Override
+        public Tuple3<Object, Double[], Map<Object, Double>[]> merge(
+                Tuple3<Object, Double[], Map<Object, Double>[]> acc0,
+                Tuple3<Object, Double[], Map<Object, Double>[]> acc1) {
+            if (acc0.f0.equals(new Object())) {
+                acc0.f0 = acc1.f0;
+            }
+            for (int i = 0; i < featureSize; i++) {
+                acc0.f1[i] += acc1.f1[i];
+                if (acc1.f2[i] != null) {
+                    acc0.f2[i] = acc1.f2[i];
+                }
+            }
+
+            return acc0;
+        }
+    }
+
+    /**
+     * Function to generate Naive Bayes model data.
+     */
+    private static class GenerateModelFunction implements AllWindowFunction<
+            Tuple3<Object, Double[], Map<Object, Double>[]>,
+            NaiveBayesModelData,
+            TimeWindow
+            > {
+        private final int featureSize;
+        private final double smoothing;
+        private final String[] featureColNames;
+
+        GenerateModelFunction(double smoothing, String[] featureColNames) {
+            this.smoothing = smoothing;
+            this.featureColNames = featureColNames;
+            this.featureSize = featureColNames.length;
+        }
+
+        @Override
+        public void apply(TimeWindow timeWindow,
+                          Iterable<Tuple3<Object, Double[], Map<Object, 
Double>[]>> values,
+                          Collector<NaiveBayesModelData> out) {
+            double[] numDocs = new double[featureSize];
+            ArrayList <Tuple3 <Object, Double[], Map <Object, Double>[]>> 
modelArray = new ArrayList <>();
+            HashSet <Object>[] categoryNumbers = new HashSet[featureSize];

Review comment:
       Can you remove the dummy space here? Same for other code.




-- 
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