You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by JonathanH5 <gi...@git.apache.org> on 2015/09/21 13:47:20 UTC

[GitHub] flink pull request: Pull Request

GitHub user JonathanH5 opened a pull request:

    https://github.com/apache/flink/pull/1156

    Pull Request

    This pull request is related to [FLINK-1719](https://issues.apache.org/jira/browse/FLINK-1719).
    Multinomial Naive Bayes was successfully implemented @tillrohrmann and different ideas proposed by other authors were incorporated.
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/JonathanH5/flink pullrequest

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/1156.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1156
    
----
commit aec4cf0b378247e479991f5356f169703ab8ee45
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-05-10T21:06:27Z

    Added a first version of the Naive Bayes Classifier, it works.

commit d216a71edcf525b70ce76310ec122b1dcd72c6c6
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-08T16:11:43Z

    First steps to convert to new MLL done

commit ab43bb2686afeed31793a4018b40e26a52c8d4c4
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-09T15:14:36Z

    Small changes for talk with Till

commit b5952dbf222ccbb18c96d6ab626236fe1505e203
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-14T09:35:57Z

    NaiveB now working with new MLL Layout

commit 1e8dfdf7494f5f57289551a434b400a92f35edb3
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-15T15:52:25Z

    Removal of old code

commit 2bcee72bf185fde3e32dbb8e3fb0bc3c8fa73c05
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-16T13:53:29Z

    Renamed to MultinomialNaiveBayes, improved code comments, created class for automatic benchmarking: MultinomalNaiveBayesRuns

commit ddadbb0299f9116bf0c2acb6b11c5f26a4bd9e10
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-22T14:55:41Z

    Added the first two possipilities to choose from, enhanced code comments and code structure

commit e8f5b7dd1496edd235b6b40d5462077059002ebd
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-23T10:45:10Z

    Added Possibility 3 and improved code comments by a lot (kind of done)

commit f7af3e06c95d58b38edae827974b182082b3d22a
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-23T12:44:41Z

    Added tests for all possibilities that use data provided by the Collection class

commit 42921e9c911fae996bac2aee4dd32a6b0ee7d3e7
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-24T09:51:22Z

    Duplicated MultinomialNaiveBayes class and renamed it to MultinomialNaiveBayesJoinedModel. Both classes (end tests) do now exactly the same (also same line numbers), only the name differs).

commit a5e9c80c2214c178f3ca7c87b6e9e763409f90e0
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-24T11:36:09Z

    MultinomialNaiveBayes now stores its data in two different models -> class related and word related, results are the same but it seems to be faster than MultinomialNaiveBayesJoinedModel, tests already work

commit a8e62cfc46eae87aece575896ea494c02bc48a11
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-07-27T12:04:35Z

    Resolved 404 Scala style errors

commit c90b25d1f8de933400a6a69c307f28cbec317bb5
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-11T22:52:49Z

    First incorparation of SR1, only Schneider so far, works but test show that accucary for webkb is 10 percent worse

commit 8c15e7c0bc8a014840baa866b51b62edce2846ae
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-14T15:53:00Z

    Added SR1 = 2, results seem weird. Also added first code for a Transformer that applies feature selection

commit bb46a2951501d9ecbdb3161c306177fde751e770
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-19T14:09:10Z

    Improved CRQ and some other things

commit 32c05d8d8860205e4e81f05a193980958a69d1b8
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-19T14:12:48Z

    Removed changes for SR1 = 2 from the Fit Operation because nothing needs to be changed there

commit ed843c6a95429a8522d71436c97f1ee0a7c8b159
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-20T11:21:02Z

    Added SR=1

commit 509184692f4352e5d228897bfde8564a35163d39
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-21T15:28:49Z

    Added R1

commit 98d1dee42c77d73e5da32246c6d7bbf9c8ac6f2e
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-25T12:44:26Z

    Resolved systematic error when calculation SR1=1, SR1=2 and R1=1

commit dd4acacb18e01aa44e708d84724a51c96a705872
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-27T11:50:25Z

    Version I used for testing the theory improvements

commit 06534f0d517219981577f678f8668c90be81bdab
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-08-28T10:55:03Z

    Small changes

commit bab1ecb076e56b8227af84303e22e4beb6751e5c
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-09-21T11:15:49Z

    Join with Huge

commit 2c70bf41f5d5f2e9536ce52995cae1e294776347
Author: Jonathan Hasenburg <jo...@campus.tu-berlin.de>
Date:   2015-09-21T11:22:47Z

    Cleanup for pull request

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-217094415
  
    Not since three months ago. I still wait for your review.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40441936
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    --- End diff --
    
    move the map to next line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40443549
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    --- End diff --
    
    You're broadcasting a set. At some point, the following code will be there in `ml/package.scala`
    ```scala
    def mapWithBcSet[B, O: TypeInformation: ClassTag](
        broadcastVariable: DataSet[B])(
        fun: (T, Seq[B]) => O)
      : DataSet[O] = {
      dataSet.map(new BroadcastSetMapper[T, B, O](dataSet.clean(fun)))
      .withBroadcastSet(broadcastVariable, "broadcastVariable")
    }
    
    private class BroadcastSetMapper[T, B, O](fun: (T, Seq[B]) => O)
      extends RichMapFunction[T, O] {
    
      var broadcastVariable: Seq[B] = _
    
      @throws(classOf[Exception])
      override def open(configuration: Configuration): Unit = {
        broadcastVariable = getRuntimeContext
          .getBroadcastVariable[B]("broadcastVariable")
          .asScala
          .toSeq
      }
    
      override def map(value: T): O = {
        fun(value, broadcastVariable)
      }
    }
    ```
    You can include this in your code or copy this file: https://raw.githubusercontent.com/sachingoel0101/flink/4e4e10259a48f3bbcbcd48410fd01d45d4882f2c/flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/package.scala
    
    This will provide a much neater way to broadcast sets.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40441800
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    --- End diff --
    
    Add documentation for the setter functions, or as in above comment, describe the parameters in class documentation. Have a look at `SVM` class documentation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-176168967
  
    Ok, I think I incorporated everything that you asked for. I will squash all of the commits and rebase everything when you give me the go for that :).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964553
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    +
    +  val outputFolder = "tmp/"
    +
    +
    +  behavior of "The MultinomialNaiveBayes implementation"
    +
    +
    +  it should "train the classifier with the basic configuration" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes()
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "basicConfigModelWord.csv", outputFolder +
    +      "basicConfigModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the basicConfigModel model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +    val nnb = MultinomialNaiveBayes()
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/basicConfigModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder +
    +        "/basicConfigModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "basicConfigSolution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "validate, that basicConfig predicted everything but 357 and 358 correctly" in {
    +    val basicA = scala.io.Source.fromFile(outputFolder + "basicConfigSolution.csv").getLines()
    +      .toArray
    +    for (line <- basicA) {
    +      val split = line.split("\t")
    +      if (split(0).toInt <= 10) {
    +        assert(split(1).equals("business"))
    +      } else if (split(0).toInt <= 356 || split(0).toInt >= 359) {
    +        assert(split(1).equals("entertainment"))
    +      } else if (split(0).toInt == 357 || split(0).toInt == 358) {
    +        assert(split(1).equals("business")) //wrong predicted, but we want this
    +      } else {
    +        fail("unknown identifier number in basicConfigSolution.csv" + split(0))
    +      }
    +    }
    +
    +  }
    +
    +  it should "train the classifier with p1 = 0" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(0)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_0ModelWord.csv", outputFolder + "p1_0ModelClass.csv")
    +
    +    env.execute()
    --- End diff --
    
    What about verification of the result?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442693
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    --- End diff --
    
    line break


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r41842185
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    --- End diff --
    
    It should follow this syntax:
    ```scala
    val result = input.mapWithBcSet(
      (data, broadcast) => {
        // broadcast is a Seq[B]
      }
    )
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442440
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    --- End diff --
    
    line break


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40444821
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    +                                predictParameters: ParameterMap,
    +                                input: DataSet[(Int, String)]): DataSet[(Int, String)] = {
    +
    +      if (instance.wordRelatedModelData.isEmpty || instance.classRelatedModelData.isEmpty) {
    +        throw new RuntimeException("The NormalNaiveBayes has not been fitted to the " +
    +            "data. This is necessary before a prediction on other data can be made.")
    +      }
    +
    +      val wordRelatedModelData = instance.wordRelatedModelData.get
    +      val classRelatedModelData = instance.classRelatedModelData.get
    +
    +      val resultingParameters = instance.parameters ++ predictParameters
    +
    +      //split the texts from the input data set into its words
    +      val words: DataSet[(Int, String)] = input.flatMap {
    +        pair => pair._2.split(" ").map { word => (pair._1, word)}
    +      }
    +
    +      //genreate word counts for each word with a key
    +      // 1. Map: put a 1 to each key
    +      // 2. Group-Reduce: group by id and word and sum the 1s
    +      val wordsAndCount: DataSet[(Int, String, Int)] = words.map(line => (line._1, line._2, 1))
    +        .groupBy(0, 1).sum(2) // (id -> word -> word count in text)
    +
    +      //calculate the count of all words for a text identified by its key
    +      val wordsInText: DataSet[(Int, Int)] = wordsAndCount.map(line => (line._1, line._3))
    +        .groupBy(0).sum(1) //(id -> all words in text)
    +
    +      //generate a data set containing all words that are in model for each id, class pair
    +      // 1. Join: wordRelatedModelData with wordsAndCount on
    +      //  words (id -> class -> word ->  word count -> log(P(w|c))
    +      val foundWords: DataSet[(Int, String, String, Int, Double)] = wordRelatedModelData
    +        .joinWithHuge(wordsAndCount).where(1).equalTo(1) {
    +        (wordR, wordsAC) => (wordsAC._1, wordR._1, wordsAC._2, wordsAC._3, wordR._3)
    +      }
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +      //RENNIE 1: transform document frequency
    +
    +      val sr1 = resultingParameters(SR1)
    +      val r1 = resultingParameters(R1)
    +      var improvementData: DataSet[(String, Double)] = null
    +
    +      if (r1 == 1) {
    +        //The improvementData data set is needed
    +        if (instance.improvementData.isEmpty) {
    +          throw new RuntimeException("R1 = 1, for that additional data is needed, but it was not" +
    +            "found. Make sure to set R1 = 1 when fitting the training data.")
    +        }
    +        improvementData = instance.improvementData.get
    +      }
    +
    +      if (sr1 == 1 && r1 == 1) {
    +        throw new RuntimeException("Parameter sr1 and r1 are both set to 1, which is not allowed.")
    +      }
    +
    +      var sumPwcFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumpwc for found words
    +        // 1. Map: Remove unneded information from foundWords and calculate the sumP(w|c) for each
    +        //  word (id -> class -> word count * log(P(w|c))
    +        // 2. Group-Reduce: on id and class, sum all (word count * log(P(w|c))) results
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4 * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 1 && r1 == 0) {
    +        //same as sr1 == 0, but there is no multiplication with the word counts
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 2 && r1 == 0) {
    +        //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, Math.log(line._4 + 1) * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 0 && r1 == 1) {
    +        //same as r1 = 0, but the word frequency is multiplied with with log (n_d(c) / n_d(w_t))
    +        //for that a join with the improvementData data set must be performed first to get the
    +        //needed additional information.
    +        // Join: (id -> class ->  word count * log P(w|c) * log (number of documents in class /
    +                                                            // word frequency in all classes)
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, found._4 * found._5 * imp._2)
    +          }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      } else if (sr1 == 2 && r1 == 1) {
    +        //combination of r1 = 1 and sr1 =2
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, Math.log(found._4 + 1) * found._5 * imp._2)
    +        }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      }
    +
    +      var sumPwcNotFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumwpc for words that are not in model in that class
    +        // 1. Map: Discard word and log(P(w|c) from foundWords
    +        // 2. Group-Reduce: calculate sum count of found words for each document,
    +        //  class pair (id -> class -> sum(wordCount))
    +        // 3. Join: with wordsInText on id, to get the count of all words per document
    +        // 4. Map: calculate sumPWcNotFound (id -> class ->
    +        //  (all words in document - found word in document) *
    +        //  log(P(w|c) not in class (provided by broadcast))
    +        sumPwcNotFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4))
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +          .join(wordsInText).where(0).equalTo(0) {
    +          (foundW, wordsIT) => (foundW._1, foundW._2, foundW._3, wordsIT._2)
    +        }.map(new RichMapFunction[(Int, String, Int, Int), (Int, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Double] = mutable
    +            .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +              .asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._3)
    +            }
    +          }
    +
    +          override def map(value: (Int, String, Int, Int)): (Int, String, Double) = {
    +            (value._1, value._2, (value._4 - value._3) * broadcastMap(value._2))
    +          }
    +        }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +      } else {
    +        /** if the word frequency is changed (as in SR1 = 1, SR1 = 2, R1 = 1), the
    +          * sumPwcNotFoundWords can not be calculated as above. They must be calculated the same way
    +          * as for the sumPwcFoundWords data set (because each word frequency for each word is
    +          * important).
    +          *
    +          * Prepare foundWords data set for differenz
    +          * 1. Map: discard log(P(w|c) and wordCount from foundWords (id -> class -> word)
    +          * = all words that are known in each document for each class
    +          *
    +          * Prepare wordsAndCounts data set for difference
    +          * 1. FlatMap:create for every tuple a (document id -> class -> word -> word count)
    +          *    tuple for every class
    +          *
    +          * Create notFoundWords data set
    +          * 1: CoGroup: Only tuples (id, class, word) that are not in preparedFoundWords
    +          *
    +          * The result is a data set, that contains all words for each document for each class
    +          * that are not part of that class and the word counts.
    +          *
    +          * Then calcualte sumPwcNotfoundWords
    +          */
    +
    +        val preparedFoundWords: DataSet[(Int, String, String)] = foundWords
    +          .map(line => (line._1, line._2, line._3))
    +
    +        val wordsAndCountsExtended: DataSet[(Int, String, String, Int)] = wordsAndCount
    +          .flatMap(new Extender())
    +          .withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +        val notFoundWords: DataSet[(Int, String, String, Int)] = wordsAndCountsExtended
    +          .coGroup(preparedFoundWords).where(0, 1, 2)
    +          .equalTo(0, 1, 2)(new DifferenceCoGrouper) //(id -> class -> word -> word count)
    +
    +        if ((sr1 == 1 && r1 == 0) || (sr1 == 2 && r1 == 0)) {
    +          //calculate the sum of all Pwc for every not found word (id -> class -> sumPwcNotFound)
    +          // 1. Map: calculate the pwc value for every word (id -> class -> pwc)
    +          // 2. Sum: Sum these pwc values for each class and document
    +          sumPwcNotFoundWords = notFoundWords
    +            .map(new RichMapFunction[(Int, String, String, Int), (Int, String, Double)] {
    +
    +            var broadcastMap: mutable.Map[String, Double] = mutable
    +              .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +            override def open(config: Configuration): Unit = {
    +              val collection = getRuntimeContext
    +                .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +                .asScala
    +              for (record <- collection) {
    +                broadcastMap.put(record._1, record._3)
    +              }
    +            }
    +
    +            override def map(value: (Int, String, String, Int)): (Int, String, Double) = {
    +              if (sr1 == 1 && r1 == 0) {
    +                //same as sr1 == 0, but there is no multiplication with the word counts
    +                return (value._1, value._2, broadcastMap(value._2))
    +              } else if (sr1 == 2 && r1 == 0) {
    +                //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +                return (value._1, value._2, Math.log(value._4 + 1) * broadcastMap(value._2))
    +              }
    +              throw new RuntimeException("sumPwcNotFound could not be calculated because you" +
    +                "choosed a not allowed parameter combination.")
    +            }
    +          }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +            .groupBy(0, 1).sum(2)
    +        } else if ((sr1 == 0 && r1 == 1) || (sr1 == 2 && r1 == 1)) {
    +
    +          //same as r1 = 0, but the word frequency is multiplied with
    +          //log (n_d(c) / n_d(w_t)) for that a join with the improvementData data set must be
    +          //performed first to get the needed additional information.
    +          // 1. Join with improvement data: (id -> class ->  word count ->
    +            // log (number of documents in class / word frequency in all classes))
    +          // 2a. Map: (id -> class -> word count * log (see above) * log(P(w|c))
    +          // 2b. Map: (id -> class -> log(word count + 1) * log (see above) * log(P(w|c))
    +          sumPwcNotFoundWords = notFoundWords
    +            .join(improvementData).where(2).equalTo(0) {
    +            (nf, imp) => (nf._1, nf._2, nf._4, imp._2)
    +          }.map(new RichMapFunction[(Int, String, Int, Double), (Int, String, Double)] {
    +
    +            var broadcastMap: mutable.Map[String, Double] = mutable
    +              .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +            override def open(config: Configuration): Unit = {
    +              val collection = getRuntimeContext
    +                .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +                .asScala
    +              for (record <- collection) {
    +                broadcastMap.put(record._1, record._3)
    +              }
    +            }
    +
    +            override def map(value: (Int, String, Int, Double)): (Int, String, Double) = {
    +              if (sr1 == 0 && r1 == 1) {
    +                return (value._1, value._2, value._3 * value._4 * broadcastMap(value._2))
    +              } else if (sr1 == 2 && r1 == 1) {
    +                return (value._1, value._2, Math.log(value._3 + 1)
    +                  * value._4 * broadcastMap(value._2))
    +              }
    +              throw new RuntimeException("sumPwcNotFound could not be calculated because you" +
    +                "choosed a not allowed parameter combination.")
    +            }
    +          }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +            .groupBy(0, 1).sum(2) //(id -> class -> sum(log(P(w|c))
    +        }
    +      }
    +
    +      var sumPwc: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //sum those pwc sums
    +        // 1. Join sumPwcFoundWords and sumPwcNotFoundWords
    +        // 2. Map: add sums from found words and sums from not found words
    +        sumPwc = sumPwcFoundWords
    +          .join(sumPwcNotFoundWords).where(0, 1).equalTo(0, 1) {
    +          (found, notfound) => (found._1, found._2, found._3 + notfound._3)
    +        } //(id -> class name -> sum log(p(w|c)))
    +      } else {
    +        //sum those pwc sums
    +        // 1. CoGroup Found with NotFound on document id
    +        // add values together, when both have a same class
    +        // submit tuple, when a class misses for the other one
    +        sumPwc = sumPwcFoundWords.coGroup(sumPwcNotFoundWords)
    +          .where(0).equalTo(0) {
    +          (foundVs, notFoundVs, out: Collector[(Int, String, Double)]) =>
    +            //1. Create two Maps
    +            var foundMap = Map[String, Double]()
    +            var notFoundMap = Map[String, Double]()
    +
    +            var currentID = -1
    +
    +            for (foundV <- foundVs) {
    +              if (currentID == -1) {
    +                currentID = foundV._1
    +              }
    +              foundMap += foundV._2 -> foundV._3
    +            }
    +            for (notFoundV <- notFoundVs) {
    +              notFoundMap += notFoundV._2 -> notFoundV._3
    +            }
    +            //2. Go through these maps and collect tuples
    +            for (tuple <- foundMap) {
    +              if (notFoundMap.contains(tuple._1)) {
    +                out.collect(currentID, tuple._1, tuple._2 + notFoundMap(tuple._1))
    +                notFoundMap.remove(tuple._1)
    +              } else {
    +                out.collect(currentID, tuple._1, tuple._2)
    +              }
    +            }
    +            for (tuple <- notFoundMap) {
    +              if (foundMap.contains(tuple._1)) {
    +                out.collect(currentID, tuple._1, tuple._2 + foundMap(tuple._1))
    +              } else {
    +                out.collect(currentID, tuple._1, tuple._2)
    +              }
    +            }
    +
    +
    +        }
    +      }
    +
    +      //END RENNIE 1
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //SCHNEIDER 1: ignore P(c_j) in cMAP formula
    +      val s1 = resultingParameters(S1)
    +      var posterior: DataSet[(Int, String, Double)] = null
    +      if (s1 == 0) {
    +        //calculate posterior values for each class
    +        // 1. Map: add sumPwc values with log(P(c)) (provided by broadcast)
    +        posterior = sumPwc
    +          .map(new RichMapFunction[(Int, String, Double), (Int, String, Double)] {
    --- End diff --
    
    Use a `mapWithBcSet` instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40538554
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    --- End diff --
    
    is it fine that way?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964537
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    +
    +  val outputFolder = "tmp/"
    +
    +
    +  behavior of "The MultinomialNaiveBayes implementation"
    +
    +
    +  it should "train the classifier with the basic configuration" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes()
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "basicConfigModelWord.csv", outputFolder +
    +      "basicConfigModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the basicConfigModel model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +    val nnb = MultinomialNaiveBayes()
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/basicConfigModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder +
    +        "/basicConfigModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "basicConfigSolution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "validate, that basicConfig predicted everything but 357 and 358 correctly" in {
    --- End diff --
    
    This is probably better a unit test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40444357
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    +                                predictParameters: ParameterMap,
    +                                input: DataSet[(Int, String)]): DataSet[(Int, String)] = {
    +
    +      if (instance.wordRelatedModelData.isEmpty || instance.classRelatedModelData.isEmpty) {
    +        throw new RuntimeException("The NormalNaiveBayes has not been fitted to the " +
    +            "data. This is necessary before a prediction on other data can be made.")
    +      }
    +
    +      val wordRelatedModelData = instance.wordRelatedModelData.get
    +      val classRelatedModelData = instance.classRelatedModelData.get
    +
    +      val resultingParameters = instance.parameters ++ predictParameters
    +
    +      //split the texts from the input data set into its words
    +      val words: DataSet[(Int, String)] = input.flatMap {
    +        pair => pair._2.split(" ").map { word => (pair._1, word)}
    +      }
    +
    +      //genreate word counts for each word with a key
    +      // 1. Map: put a 1 to each key
    +      // 2. Group-Reduce: group by id and word and sum the 1s
    +      val wordsAndCount: DataSet[(Int, String, Int)] = words.map(line => (line._1, line._2, 1))
    +        .groupBy(0, 1).sum(2) // (id -> word -> word count in text)
    +
    +      //calculate the count of all words for a text identified by its key
    +      val wordsInText: DataSet[(Int, Int)] = wordsAndCount.map(line => (line._1, line._3))
    +        .groupBy(0).sum(1) //(id -> all words in text)
    +
    +      //generate a data set containing all words that are in model for each id, class pair
    +      // 1. Join: wordRelatedModelData with wordsAndCount on
    +      //  words (id -> class -> word ->  word count -> log(P(w|c))
    +      val foundWords: DataSet[(Int, String, String, Int, Double)] = wordRelatedModelData
    +        .joinWithHuge(wordsAndCount).where(1).equalTo(1) {
    +        (wordR, wordsAC) => (wordsAC._1, wordR._1, wordsAC._2, wordsAC._3, wordR._3)
    +      }
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +      //RENNIE 1: transform document frequency
    +
    +      val sr1 = resultingParameters(SR1)
    +      val r1 = resultingParameters(R1)
    +      var improvementData: DataSet[(String, Double)] = null
    +
    +      if (r1 == 1) {
    +        //The improvementData data set is needed
    +        if (instance.improvementData.isEmpty) {
    +          throw new RuntimeException("R1 = 1, for that additional data is needed, but it was not" +
    +            "found. Make sure to set R1 = 1 when fitting the training data.")
    +        }
    +        improvementData = instance.improvementData.get
    +      }
    +
    +      if (sr1 == 1 && r1 == 1) {
    +        throw new RuntimeException("Parameter sr1 and r1 are both set to 1, which is not allowed.")
    +      }
    +
    +      var sumPwcFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumpwc for found words
    +        // 1. Map: Remove unneded information from foundWords and calculate the sumP(w|c) for each
    +        //  word (id -> class -> word count * log(P(w|c))
    +        // 2. Group-Reduce: on id and class, sum all (word count * log(P(w|c))) results
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4 * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 1 && r1 == 0) {
    +        //same as sr1 == 0, but there is no multiplication with the word counts
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 2 && r1 == 0) {
    +        //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, Math.log(line._4 + 1) * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 0 && r1 == 1) {
    +        //same as r1 = 0, but the word frequency is multiplied with with log (n_d(c) / n_d(w_t))
    +        //for that a join with the improvementData data set must be performed first to get the
    +        //needed additional information.
    +        // Join: (id -> class ->  word count * log P(w|c) * log (number of documents in class /
    +                                                            // word frequency in all classes)
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, found._4 * found._5 * imp._2)
    +          }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      } else if (sr1 == 2 && r1 == 1) {
    +        //combination of r1 = 1 and sr1 =2
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, Math.log(found._4 + 1) * found._5 * imp._2)
    +        }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      }
    +
    +      var sumPwcNotFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumwpc for words that are not in model in that class
    +        // 1. Map: Discard word and log(P(w|c) from foundWords
    +        // 2. Group-Reduce: calculate sum count of found words for each document,
    +        //  class pair (id -> class -> sum(wordCount))
    +        // 3. Join: with wordsInText on id, to get the count of all words per document
    +        // 4. Map: calculate sumPWcNotFound (id -> class ->
    +        //  (all words in document - found word in document) *
    +        //  log(P(w|c) not in class (provided by broadcast))
    +        sumPwcNotFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4))
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +          .join(wordsInText).where(0).equalTo(0) {
    +          (foundW, wordsIT) => (foundW._1, foundW._2, foundW._3, wordsIT._2)
    +        }.map(new RichMapFunction[(Int, String, Int, Int), (Int, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Double] = mutable
    +            .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +              .asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._3)
    +            }
    +          }
    +
    +          override def map(value: (Int, String, Int, Int)): (Int, String, Double) = {
    +            (value._1, value._2, (value._4 - value._3) * broadcastMap(value._2))
    +          }
    +        }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +      } else {
    +        /** if the word frequency is changed (as in SR1 = 1, SR1 = 2, R1 = 1), the
    +          * sumPwcNotFoundWords can not be calculated as above. They must be calculated the same way
    +          * as for the sumPwcFoundWords data set (because each word frequency for each word is
    +          * important).
    +          *
    +          * Prepare foundWords data set for differenz
    +          * 1. Map: discard log(P(w|c) and wordCount from foundWords (id -> class -> word)
    +          * = all words that are known in each document for each class
    +          *
    +          * Prepare wordsAndCounts data set for difference
    +          * 1. FlatMap:create for every tuple a (document id -> class -> word -> word count)
    +          *    tuple for every class
    +          *
    +          * Create notFoundWords data set
    +          * 1: CoGroup: Only tuples (id, class, word) that are not in preparedFoundWords
    +          *
    +          * The result is a data set, that contains all words for each document for each class
    +          * that are not part of that class and the word counts.
    +          *
    +          * Then calcualte sumPwcNotfoundWords
    +          */
    +
    +        val preparedFoundWords: DataSet[(Int, String, String)] = foundWords
    +          .map(line => (line._1, line._2, line._3))
    +
    +        val wordsAndCountsExtended: DataSet[(Int, String, String, Int)] = wordsAndCount
    +          .flatMap(new Extender())
    +          .withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +        val notFoundWords: DataSet[(Int, String, String, Int)] = wordsAndCountsExtended
    +          .coGroup(preparedFoundWords).where(0, 1, 2)
    +          .equalTo(0, 1, 2)(new DifferenceCoGrouper) //(id -> class -> word -> word count)
    +
    +        if ((sr1 == 1 && r1 == 0) || (sr1 == 2 && r1 == 0)) {
    +          //calculate the sum of all Pwc for every not found word (id -> class -> sumPwcNotFound)
    +          // 1. Map: calculate the pwc value for every word (id -> class -> pwc)
    +          // 2. Sum: Sum these pwc values for each class and document
    +          sumPwcNotFoundWords = notFoundWords
    +            .map(new RichMapFunction[(Int, String, String, Int), (Int, String, Double)] {
    +
    +            var broadcastMap: mutable.Map[String, Double] = mutable
    --- End diff --
    
    should be on the same line


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40802836
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    --- End diff --
    
    It makes the code more scala-esque and cleaner. The reason it's not in right now is because it wasn't needed till now. :-')


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by uce <gi...@git.apache.org>.
Github user uce commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-141955459
  
    Hey Jonathan, could you change the title of this pull request to match the JIRA issue it relates to? A better title could be: [FLINK-1719] [ml] Add naive Bayes classification algorithm.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40441863
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    --- End diff --
    
    indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-220260011
  
    Sorry, currently I don't have the time to review your PR. If nobody else steps up, it will probably still take some time.
    
    @danielblazevski you're right, ideally the naive Bayes implementation works with textual data as well as numerical data.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by danielblazevski <gi...@git.apache.org>.
Github user danielblazevski commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-220201194
  
    Should the input be DataSets of Strings?  The documentation, syntax are heavy on the side of dealing exclusively with text classification.  Could just be me since don't come from an NLP background, and this made is less clear to read the logic of the code, e.g. I'm more used to applying numeric values for Bayes.  See, e.g. examples of using Bayes in scikit-learn: http://scikit-learn.org/stable/modules/generated/sklearn.naive_bayes.GaussianNB.html.  I presume the goal of this issue would be to apply to contexts outside of text classification and allow for users to apply the Bayes to numeric data like the scikit-learn example.  Correct me if I'm wrong @tillrohrmann @chiwanpark 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-142014791
  
    How about closing this pull request, doing some cleanup, a good name, and reopening a new one?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-220265510
  
    I hope that this can be reviewed in the near future. @danielblazevski it might be nicer if numerical data can be processed as well, but I think a version that can process textual data is still better than nothing at the moment, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442722
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    --- End diff --
    
    line breaks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964615
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    +
    +  val outputFolder = "tmp/"
    +
    +
    +  behavior of "The MultinomialNaiveBayes implementation"
    +
    +
    +  it should "train the classifier with the basic configuration" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes()
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "basicConfigModelWord.csv", outputFolder +
    +      "basicConfigModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the basicConfigModel model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +    val nnb = MultinomialNaiveBayes()
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/basicConfigModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder +
    +        "/basicConfigModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "basicConfigSolution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "validate, that basicConfig predicted everything but 357 and 358 correctly" in {
    +    val basicA = scala.io.Source.fromFile(outputFolder + "basicConfigSolution.csv").getLines()
    +      .toArray
    +    for (line <- basicA) {
    +      val split = line.split("\t")
    +      if (split(0).toInt <= 10) {
    +        assert(split(1).equals("business"))
    +      } else if (split(0).toInt <= 356 || split(0).toInt >= 359) {
    +        assert(split(1).equals("entertainment"))
    +      } else if (split(0).toInt == 357 || split(0).toInt == 358) {
    +        assert(split(1).equals("business")) //wrong predicted, but we want this
    +      } else {
    +        fail("unknown identifier number in basicConfigSolution.csv" + split(0))
    +      }
    +    }
    +
    +  }
    +
    +  it should "train the classifier with p1 = 0" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(0)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_0ModelWord.csv", outputFolder + "p1_0ModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the p1 = 0 model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +
    +    val nnb = MultinomialNaiveBayes().setP2(0)
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/p1_0ModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder + "/p1_0ModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "p1_0Solution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "train the classifier with p1 = 1" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(1)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_1ModelWord.csv", outputFolder + "p1_1ModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the p1 = 1 model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +
    +    val nnb = MultinomialNaiveBayes().setP2(1)
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/p1_1ModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder + "/p1_1ModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "p1_1Solution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +
    +    env.execute()
    +
    --- End diff --
    
    line break


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442383
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    --- End diff --
    
    same as above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442397
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    --- End diff --
    
    line break


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40441149
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    --- End diff --
    
    The value should be in the same line as the declaration IMO. You can put the comment before that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964749
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[MultinomialNaiveBayes,
    +    (Int, String), (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    --- End diff --
    
    Formatting of parameter list: Every parameter should be in a single line if the complete list does not fit in a single line. Take a look at the other Scala code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40441437
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    --- End diff --
    
    This should be written like
    ```scala
    def setModelDataSet(
          wordRelated : DataSet[(String, String, Double)],
          classRelated: DataSet[(String, Double, Double)])
        : Unit = {
        this.wordRelatedModelData = Some(wordRelated)
        this.classRelatedModelData = Some(classRelated)
      }
    ```
    Have a look at the existing code for indentation conventions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442852
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    --- End diff --
    
    Move to separate line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by rmetzger <gi...@git.apache.org>.
Github user rmetzger commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-141967992
  
    The commit history needs a cleanup


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-220069001
  
    @tillrohrmann : Any news regarding the review? It is pending now a long time and GitHub tells me know that there are conflicts.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40443743
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    --- End diff --
    
    indentations.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by chobeat <gi...@git.apache.org>.
Github user chobeat commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-217093665
  
    any news?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442340
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    --- End diff --
    
    map should be on the next line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-143339059
  
    Hi @sachingoel0101, I will fix the styling issues... . Thx.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964582
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    +
    +  val outputFolder = "tmp/"
    +
    +
    +  behavior of "The MultinomialNaiveBayes implementation"
    +
    +
    +  it should "train the classifier with the basic configuration" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes()
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "basicConfigModelWord.csv", outputFolder +
    +      "basicConfigModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the basicConfigModel model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +    val nnb = MultinomialNaiveBayes()
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/basicConfigModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder +
    +        "/basicConfigModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "basicConfigSolution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "validate, that basicConfig predicted everything but 357 and 358 correctly" in {
    +    val basicA = scala.io.Source.fromFile(outputFolder + "basicConfigSolution.csv").getLines()
    +      .toArray
    +    for (line <- basicA) {
    +      val split = line.split("\t")
    +      if (split(0).toInt <= 10) {
    +        assert(split(1).equals("business"))
    +      } else if (split(0).toInt <= 356 || split(0).toInt >= 359) {
    +        assert(split(1).equals("entertainment"))
    +      } else if (split(0).toInt == 357 || split(0).toInt == 358) {
    +        assert(split(1).equals("business")) //wrong predicted, but we want this
    +      } else {
    +        fail("unknown identifier number in basicConfigSolution.csv" + split(0))
    +      }
    +    }
    +
    +  }
    +
    +  it should "train the classifier with p1 = 0" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(0)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_0ModelWord.csv", outputFolder + "p1_0ModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the p1 = 0 model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +
    +    val nnb = MultinomialNaiveBayes().setP2(0)
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/p1_0ModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder + "/p1_0ModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "p1_0Solution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +
    +    env.execute()
    +
    --- End diff --
    
    line break


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40541853
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    --- End diff --
    
    Why do you think is it a good idea to use ml/package.scala if it is not part of the official Flink version? There has to be a reason why it is not supported by know... . I suggest to keep it how it is right know and wait until the new broadcast logic is officially introduced...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40443013
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    --- End diff --
    
    indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40444866
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    +                                predictParameters: ParameterMap,
    +                                input: DataSet[(Int, String)]): DataSet[(Int, String)] = {
    +
    +      if (instance.wordRelatedModelData.isEmpty || instance.classRelatedModelData.isEmpty) {
    +        throw new RuntimeException("The NormalNaiveBayes has not been fitted to the " +
    +            "data. This is necessary before a prediction on other data can be made.")
    +      }
    +
    +      val wordRelatedModelData = instance.wordRelatedModelData.get
    +      val classRelatedModelData = instance.classRelatedModelData.get
    +
    +      val resultingParameters = instance.parameters ++ predictParameters
    +
    +      //split the texts from the input data set into its words
    +      val words: DataSet[(Int, String)] = input.flatMap {
    +        pair => pair._2.split(" ").map { word => (pair._1, word)}
    +      }
    +
    +      //genreate word counts for each word with a key
    +      // 1. Map: put a 1 to each key
    +      // 2. Group-Reduce: group by id and word and sum the 1s
    +      val wordsAndCount: DataSet[(Int, String, Int)] = words.map(line => (line._1, line._2, 1))
    +        .groupBy(0, 1).sum(2) // (id -> word -> word count in text)
    +
    +      //calculate the count of all words for a text identified by its key
    +      val wordsInText: DataSet[(Int, Int)] = wordsAndCount.map(line => (line._1, line._3))
    +        .groupBy(0).sum(1) //(id -> all words in text)
    +
    +      //generate a data set containing all words that are in model for each id, class pair
    +      // 1. Join: wordRelatedModelData with wordsAndCount on
    +      //  words (id -> class -> word ->  word count -> log(P(w|c))
    +      val foundWords: DataSet[(Int, String, String, Int, Double)] = wordRelatedModelData
    +        .joinWithHuge(wordsAndCount).where(1).equalTo(1) {
    +        (wordR, wordsAC) => (wordsAC._1, wordR._1, wordsAC._2, wordsAC._3, wordR._3)
    +      }
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +      //RENNIE 1: transform document frequency
    +
    +      val sr1 = resultingParameters(SR1)
    +      val r1 = resultingParameters(R1)
    +      var improvementData: DataSet[(String, Double)] = null
    +
    +      if (r1 == 1) {
    +        //The improvementData data set is needed
    +        if (instance.improvementData.isEmpty) {
    +          throw new RuntimeException("R1 = 1, for that additional data is needed, but it was not" +
    +            "found. Make sure to set R1 = 1 when fitting the training data.")
    +        }
    +        improvementData = instance.improvementData.get
    +      }
    +
    +      if (sr1 == 1 && r1 == 1) {
    +        throw new RuntimeException("Parameter sr1 and r1 are both set to 1, which is not allowed.")
    +      }
    +
    +      var sumPwcFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumpwc for found words
    +        // 1. Map: Remove unneded information from foundWords and calculate the sumP(w|c) for each
    +        //  word (id -> class -> word count * log(P(w|c))
    +        // 2. Group-Reduce: on id and class, sum all (word count * log(P(w|c))) results
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4 * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 1 && r1 == 0) {
    +        //same as sr1 == 0, but there is no multiplication with the word counts
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 2 && r1 == 0) {
    +        //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, Math.log(line._4 + 1) * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 0 && r1 == 1) {
    +        //same as r1 = 0, but the word frequency is multiplied with with log (n_d(c) / n_d(w_t))
    +        //for that a join with the improvementData data set must be performed first to get the
    +        //needed additional information.
    +        // Join: (id -> class ->  word count * log P(w|c) * log (number of documents in class /
    +                                                            // word frequency in all classes)
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, found._4 * found._5 * imp._2)
    +          }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      } else if (sr1 == 2 && r1 == 1) {
    +        //combination of r1 = 1 and sr1 =2
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, Math.log(found._4 + 1) * found._5 * imp._2)
    +        }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      }
    +
    +      var sumPwcNotFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumwpc for words that are not in model in that class
    +        // 1. Map: Discard word and log(P(w|c) from foundWords
    +        // 2. Group-Reduce: calculate sum count of found words for each document,
    +        //  class pair (id -> class -> sum(wordCount))
    +        // 3. Join: with wordsInText on id, to get the count of all words per document
    +        // 4. Map: calculate sumPWcNotFound (id -> class ->
    +        //  (all words in document - found word in document) *
    +        //  log(P(w|c) not in class (provided by broadcast))
    +        sumPwcNotFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4))
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +          .join(wordsInText).where(0).equalTo(0) {
    +          (foundW, wordsIT) => (foundW._1, foundW._2, foundW._3, wordsIT._2)
    +        }.map(new RichMapFunction[(Int, String, Int, Int), (Int, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Double] = mutable
    +            .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +              .asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._3)
    +            }
    +          }
    +
    +          override def map(value: (Int, String, Int, Int)): (Int, String, Double) = {
    +            (value._1, value._2, (value._4 - value._3) * broadcastMap(value._2))
    +          }
    +        }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +      } else {
    +        /** if the word frequency is changed (as in SR1 = 1, SR1 = 2, R1 = 1), the
    +          * sumPwcNotFoundWords can not be calculated as above. They must be calculated the same way
    +          * as for the sumPwcFoundWords data set (because each word frequency for each word is
    +          * important).
    +          *
    +          * Prepare foundWords data set for differenz
    +          * 1. Map: discard log(P(w|c) and wordCount from foundWords (id -> class -> word)
    +          * = all words that are known in each document for each class
    +          *
    +          * Prepare wordsAndCounts data set for difference
    +          * 1. FlatMap:create for every tuple a (document id -> class -> word -> word count)
    +          *    tuple for every class
    +          *
    +          * Create notFoundWords data set
    +          * 1: CoGroup: Only tuples (id, class, word) that are not in preparedFoundWords
    +          *
    +          * The result is a data set, that contains all words for each document for each class
    +          * that are not part of that class and the word counts.
    +          *
    +          * Then calcualte sumPwcNotfoundWords
    +          */
    +
    +        val preparedFoundWords: DataSet[(Int, String, String)] = foundWords
    +          .map(line => (line._1, line._2, line._3))
    +
    +        val wordsAndCountsExtended: DataSet[(Int, String, String, Int)] = wordsAndCount
    +          .flatMap(new Extender())
    +          .withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +        val notFoundWords: DataSet[(Int, String, String, Int)] = wordsAndCountsExtended
    +          .coGroup(preparedFoundWords).where(0, 1, 2)
    +          .equalTo(0, 1, 2)(new DifferenceCoGrouper) //(id -> class -> word -> word count)
    +
    +        if ((sr1 == 1 && r1 == 0) || (sr1 == 2 && r1 == 0)) {
    +          //calculate the sum of all Pwc for every not found word (id -> class -> sumPwcNotFound)
    +          // 1. Map: calculate the pwc value for every word (id -> class -> pwc)
    +          // 2. Sum: Sum these pwc values for each class and document
    +          sumPwcNotFoundWords = notFoundWords
    +            .map(new RichMapFunction[(Int, String, String, Int), (Int, String, Double)] {
    +
    +            var broadcastMap: mutable.Map[String, Double] = mutable
    +              .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +            override def open(config: Configuration): Unit = {
    +              val collection = getRuntimeContext
    +                .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +                .asScala
    +              for (record <- collection) {
    +                broadcastMap.put(record._1, record._3)
    +              }
    +            }
    +
    +            override def map(value: (Int, String, String, Int)): (Int, String, Double) = {
    +              if (sr1 == 1 && r1 == 0) {
    +                //same as sr1 == 0, but there is no multiplication with the word counts
    +                return (value._1, value._2, broadcastMap(value._2))
    +              } else if (sr1 == 2 && r1 == 0) {
    +                //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +                return (value._1, value._2, Math.log(value._4 + 1) * broadcastMap(value._2))
    +              }
    +              throw new RuntimeException("sumPwcNotFound could not be calculated because you" +
    +                "choosed a not allowed parameter combination.")
    +            }
    +          }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +            .groupBy(0, 1).sum(2)
    +        } else if ((sr1 == 0 && r1 == 1) || (sr1 == 2 && r1 == 1)) {
    +
    +          //same as r1 = 0, but the word frequency is multiplied with
    +          //log (n_d(c) / n_d(w_t)) for that a join with the improvementData data set must be
    +          //performed first to get the needed additional information.
    +          // 1. Join with improvement data: (id -> class ->  word count ->
    +            // log (number of documents in class / word frequency in all classes))
    +          // 2a. Map: (id -> class -> word count * log (see above) * log(P(w|c))
    +          // 2b. Map: (id -> class -> log(word count + 1) * log (see above) * log(P(w|c))
    +          sumPwcNotFoundWords = notFoundWords
    +            .join(improvementData).where(2).equalTo(0) {
    +            (nf, imp) => (nf._1, nf._2, nf._4, imp._2)
    +          }.map(new RichMapFunction[(Int, String, Int, Double), (Int, String, Double)] {
    +
    +            var broadcastMap: mutable.Map[String, Double] = mutable
    +              .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +            override def open(config: Configuration): Unit = {
    +              val collection = getRuntimeContext
    +                .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +                .asScala
    +              for (record <- collection) {
    +                broadcastMap.put(record._1, record._3)
    +              }
    +            }
    +
    +            override def map(value: (Int, String, Int, Double)): (Int, String, Double) = {
    +              if (sr1 == 0 && r1 == 1) {
    +                return (value._1, value._2, value._3 * value._4 * broadcastMap(value._2))
    +              } else if (sr1 == 2 && r1 == 1) {
    +                return (value._1, value._2, Math.log(value._3 + 1)
    +                  * value._4 * broadcastMap(value._2))
    +              }
    +              throw new RuntimeException("sumPwcNotFound could not be calculated because you" +
    +                "choosed a not allowed parameter combination.")
    +            }
    +          }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +            .groupBy(0, 1).sum(2) //(id -> class -> sum(log(P(w|c))
    +        }
    +      }
    +
    +      var sumPwc: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //sum those pwc sums
    +        // 1. Join sumPwcFoundWords and sumPwcNotFoundWords
    +        // 2. Map: add sums from found words and sums from not found words
    +        sumPwc = sumPwcFoundWords
    +          .join(sumPwcNotFoundWords).where(0, 1).equalTo(0, 1) {
    +          (found, notfound) => (found._1, found._2, found._3 + notfound._3)
    +        } //(id -> class name -> sum log(p(w|c)))
    +      } else {
    +        //sum those pwc sums
    +        // 1. CoGroup Found with NotFound on document id
    +        // add values together, when both have a same class
    +        // submit tuple, when a class misses for the other one
    +        sumPwc = sumPwcFoundWords.coGroup(sumPwcNotFoundWords)
    +          .where(0).equalTo(0) {
    +          (foundVs, notFoundVs, out: Collector[(Int, String, Double)]) =>
    +            //1. Create two Maps
    +            var foundMap = Map[String, Double]()
    +            var notFoundMap = Map[String, Double]()
    +
    +            var currentID = -1
    +
    +            for (foundV <- foundVs) {
    +              if (currentID == -1) {
    +                currentID = foundV._1
    +              }
    +              foundMap += foundV._2 -> foundV._3
    +            }
    +            for (notFoundV <- notFoundVs) {
    +              notFoundMap += notFoundV._2 -> notFoundV._3
    +            }
    +            //2. Go through these maps and collect tuples
    +            for (tuple <- foundMap) {
    +              if (notFoundMap.contains(tuple._1)) {
    +                out.collect(currentID, tuple._1, tuple._2 + notFoundMap(tuple._1))
    +                notFoundMap.remove(tuple._1)
    +              } else {
    +                out.collect(currentID, tuple._1, tuple._2)
    +              }
    +            }
    +            for (tuple <- notFoundMap) {
    +              if (foundMap.contains(tuple._1)) {
    +                out.collect(currentID, tuple._1, tuple._2 + foundMap(tuple._1))
    +              } else {
    +                out.collect(currentID, tuple._1, tuple._2)
    +              }
    +            }
    +
    +
    +        }
    +      }
    +
    +      //END RENNIE 1
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //SCHNEIDER 1: ignore P(c_j) in cMAP formula
    +      val s1 = resultingParameters(S1)
    +      var posterior: DataSet[(Int, String, Double)] = null
    +      if (s1 == 0) {
    +        //calculate posterior values for each class
    +        // 1. Map: add sumPwc values with log(P(c)) (provided by broadcast)
    +        posterior = sumPwc
    +          .map(new RichMapFunction[(Int, String, Double), (Int, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Double] =
    +            mutable.Map[String, Double]() //class -> log(P(c))
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +              .asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (Int, String, Double)): (Int, String, Double) = {
    +            (value._1, value._2, value._3 + broadcastMap(value._2))
    +          }
    +        }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +      } else if (s1 == 1) {
    +        posterior = sumPwc
    +      }
    +      //choose the highest probable class
    +      // 1. Reduce: keep only highest probability
    +      posterior.groupBy(0).reduce(new CalculateReducer())
    +        .map(line => (line._1, line._2)) //(id -> class)
    +
    +
    +    }
    +  }
    +
    +
    +  /*
    +  * ************************************************************************************************
    +  * *******************************************Function Classes*************************************
    +  * ************************************************************************************************
    +   */
    +
    +  /**
    +   * Transforms a (String, String) tuple into a (String, String, Int)) tuple.
    +   * The second string from the input gets split into its words, for each word a tuple is collected
    +   * with the Int 1.
    +   */
    +  class SingleWordSplitter() extends FlatMapFunction[(String, String), (String, String, Int)] {
    +    override def flatMap(value: (String, String), out: Collector[(String, String, Int)]): Unit = {
    +      for (token: String <- value._2.split(" ")) {
    +        out.collect((value._1, token, 1))
    +      }
    +    }
    +
    +    // Can be implemented via: input.flatMap{ pair => pair._2.split(" ")
    +    // .map{ token => (pair._1, token ,1)} }
    +  }
    +
    +  /**
    +   * Transforms a (String, String) tuple into a (String, String, Int)) tuple.
    +   * The second string from the input gets split into its words, for each distinct word a tuple
    +   * is collected with the Int 1.
    +   */
    +  class SingleDistinctWordSplitter() extends FlatMapFunction[
    +    (String, String),
    +    (String, String, Int)] {
    +    override def flatMap(value: (String, String), out: Collector[(String, String, Int)]): Unit = {
    +      var x: ListBuffer[String] = ListBuffer()
    +      for (token: String <- value._2.split(" ")) {
    +        x.append(token)
    +      }
    +      x = x.distinct
    +      for (item <- x) {
    +        out.collect((value._1, item, 1))
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Used to extend the wordsAndCount data set. Transforms (id -> word -> word count) into
    +   * (id -> class -> word -> word count) for each possible class.
    +   */
    +  class Extender() extends RichFlatMapFunction[(Int, String, Int), (Int, String, String, Int)] {
    +
    +    var broadcastSet = Set[String]()
    +
    +    override def open(config: Configuration): Unit = {
    +      val collection = getRuntimeContext
    +        .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +        .asScala
    +      for (record <- collection) {
    +        broadcastSet += record._1
    +      }
    +    }
    +
    +    override def flatMap(value: (Int, String, Int), out: Collector[(Int, String, String, Int)]):
    +      Unit = {
    +      for (c: String <- broadcastSet) {
    +        out.collect((value._1, c, value._2, value._3))
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Chooses for each label that class with the highest value
    +   */
    +  class CalculateReducer() extends ReduceFunction[(Int, String, Double)] {
    +    override def reduce(value1: (Int, String, Double),
    --- End diff --
    
    indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-142256180
  
    Any additional comments? I included the other ones... . 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442050
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    --- End diff --
    
    line break is not needed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964618
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    +
    +  val outputFolder = "tmp/"
    +
    +
    +  behavior of "The MultinomialNaiveBayes implementation"
    +
    +
    +  it should "train the classifier with the basic configuration" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes()
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "basicConfigModelWord.csv", outputFolder +
    +      "basicConfigModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the basicConfigModel model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +    val nnb = MultinomialNaiveBayes()
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/basicConfigModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder +
    +        "/basicConfigModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "basicConfigSolution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "validate, that basicConfig predicted everything but 357 and 358 correctly" in {
    +    val basicA = scala.io.Source.fromFile(outputFolder + "basicConfigSolution.csv").getLines()
    +      .toArray
    +    for (line <- basicA) {
    +      val split = line.split("\t")
    +      if (split(0).toInt <= 10) {
    +        assert(split(1).equals("business"))
    +      } else if (split(0).toInt <= 356 || split(0).toInt >= 359) {
    +        assert(split(1).equals("entertainment"))
    +      } else if (split(0).toInt == 357 || split(0).toInt == 358) {
    +        assert(split(1).equals("business")) //wrong predicted, but we want this
    +      } else {
    +        fail("unknown identifier number in basicConfigSolution.csv" + split(0))
    +      }
    +    }
    +
    +  }
    +
    +  it should "train the classifier with p1 = 0" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(0)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_0ModelWord.csv", outputFolder + "p1_0ModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the p1 = 0 model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +
    +    val nnb = MultinomialNaiveBayes().setP2(0)
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/p1_0ModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder + "/p1_0ModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "p1_0Solution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "train the classifier with p1 = 1" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(1)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_1ModelWord.csv", outputFolder + "p1_1ModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the p1 = 1 model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +
    +    val nnb = MultinomialNaiveBayes().setP2(1)
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/p1_1ModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder + "/p1_1ModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "p1_1Solution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +
    +    env.execute()
    --- End diff --
    
    verification?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964672
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +
    --- End diff --
    
    line breaks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-143260576
  
    Hey @JonathanH5 , I've had a quick look through your code. Can you fix the styling issues?
    Also, please be more consistent with your use of line breaks.
    
    I will try to review the algorithm after this. [Or someone else]


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964603
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    +
    +  val outputFolder = "tmp/"
    +
    +
    +  behavior of "The MultinomialNaiveBayes implementation"
    +
    +
    +  it should "train the classifier with the basic configuration" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes()
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "basicConfigModelWord.csv", outputFolder +
    +      "basicConfigModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the basicConfigModel model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +    val nnb = MultinomialNaiveBayes()
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/basicConfigModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder +
    +        "/basicConfigModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "basicConfigSolution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "validate, that basicConfig predicted everything but 357 and 358 correctly" in {
    +    val basicA = scala.io.Source.fromFile(outputFolder + "basicConfigSolution.csv").getLines()
    +      .toArray
    +    for (line <- basicA) {
    +      val split = line.split("\t")
    +      if (split(0).toInt <= 10) {
    +        assert(split(1).equals("business"))
    +      } else if (split(0).toInt <= 356 || split(0).toInt >= 359) {
    +        assert(split(1).equals("entertainment"))
    +      } else if (split(0).toInt == 357 || split(0).toInt == 358) {
    +        assert(split(1).equals("business")) //wrong predicted, but we want this
    +      } else {
    +        fail("unknown identifier number in basicConfigSolution.csv" + split(0))
    +      }
    +    }
    +
    +  }
    +
    +  it should "train the classifier with p1 = 0" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(0)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_0ModelWord.csv", outputFolder + "p1_0ModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the p1 = 0 model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +
    +    val nnb = MultinomialNaiveBayes().setP2(0)
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/p1_0ModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder + "/p1_0ModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "p1_0Solution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "train the classifier with p1 = 1" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(1)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_1ModelWord.csv", outputFolder + "p1_1ModelClass.csv")
    +
    +    env.execute()
    --- End diff --
    
    verification?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40443702
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    --- End diff --
    
    Follow the parameter wrapping conventions as with functions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-142208569
  
    I haven't looked in depth at the PR just superficially.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40440950
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    --- End diff --
    
    Can you add more description of the different *algorithms* instead of possibilities? Also, please include the links to the specific papers.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40444429
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    +                                predictParameters: ParameterMap,
    +                                input: DataSet[(Int, String)]): DataSet[(Int, String)] = {
    +
    +      if (instance.wordRelatedModelData.isEmpty || instance.classRelatedModelData.isEmpty) {
    +        throw new RuntimeException("The NormalNaiveBayes has not been fitted to the " +
    +            "data. This is necessary before a prediction on other data can be made.")
    +      }
    +
    +      val wordRelatedModelData = instance.wordRelatedModelData.get
    +      val classRelatedModelData = instance.classRelatedModelData.get
    +
    +      val resultingParameters = instance.parameters ++ predictParameters
    +
    +      //split the texts from the input data set into its words
    +      val words: DataSet[(Int, String)] = input.flatMap {
    +        pair => pair._2.split(" ").map { word => (pair._1, word)}
    +      }
    +
    +      //genreate word counts for each word with a key
    +      // 1. Map: put a 1 to each key
    +      // 2. Group-Reduce: group by id and word and sum the 1s
    +      val wordsAndCount: DataSet[(Int, String, Int)] = words.map(line => (line._1, line._2, 1))
    +        .groupBy(0, 1).sum(2) // (id -> word -> word count in text)
    +
    +      //calculate the count of all words for a text identified by its key
    +      val wordsInText: DataSet[(Int, Int)] = wordsAndCount.map(line => (line._1, line._3))
    +        .groupBy(0).sum(1) //(id -> all words in text)
    +
    +      //generate a data set containing all words that are in model for each id, class pair
    +      // 1. Join: wordRelatedModelData with wordsAndCount on
    +      //  words (id -> class -> word ->  word count -> log(P(w|c))
    +      val foundWords: DataSet[(Int, String, String, Int, Double)] = wordRelatedModelData
    +        .joinWithHuge(wordsAndCount).where(1).equalTo(1) {
    +        (wordR, wordsAC) => (wordsAC._1, wordR._1, wordsAC._2, wordsAC._3, wordR._3)
    +      }
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +      //RENNIE 1: transform document frequency
    +
    +      val sr1 = resultingParameters(SR1)
    +      val r1 = resultingParameters(R1)
    +      var improvementData: DataSet[(String, Double)] = null
    +
    +      if (r1 == 1) {
    +        //The improvementData data set is needed
    +        if (instance.improvementData.isEmpty) {
    +          throw new RuntimeException("R1 = 1, for that additional data is needed, but it was not" +
    +            "found. Make sure to set R1 = 1 when fitting the training data.")
    +        }
    +        improvementData = instance.improvementData.get
    +      }
    +
    +      if (sr1 == 1 && r1 == 1) {
    +        throw new RuntimeException("Parameter sr1 and r1 are both set to 1, which is not allowed.")
    +      }
    +
    +      var sumPwcFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumpwc for found words
    +        // 1. Map: Remove unneded information from foundWords and calculate the sumP(w|c) for each
    +        //  word (id -> class -> word count * log(P(w|c))
    +        // 2. Group-Reduce: on id and class, sum all (word count * log(P(w|c))) results
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4 * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 1 && r1 == 0) {
    +        //same as sr1 == 0, but there is no multiplication with the word counts
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 2 && r1 == 0) {
    +        //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, Math.log(line._4 + 1) * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 0 && r1 == 1) {
    +        //same as r1 = 0, but the word frequency is multiplied with with log (n_d(c) / n_d(w_t))
    +        //for that a join with the improvementData data set must be performed first to get the
    +        //needed additional information.
    +        // Join: (id -> class ->  word count * log P(w|c) * log (number of documents in class /
    +                                                            // word frequency in all classes)
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, found._4 * found._5 * imp._2)
    +          }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      } else if (sr1 == 2 && r1 == 1) {
    +        //combination of r1 = 1 and sr1 =2
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, Math.log(found._4 + 1) * found._5 * imp._2)
    +        }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      }
    +
    +      var sumPwcNotFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumwpc for words that are not in model in that class
    +        // 1. Map: Discard word and log(P(w|c) from foundWords
    +        // 2. Group-Reduce: calculate sum count of found words for each document,
    +        //  class pair (id -> class -> sum(wordCount))
    +        // 3. Join: with wordsInText on id, to get the count of all words per document
    +        // 4. Map: calculate sumPWcNotFound (id -> class ->
    +        //  (all words in document - found word in document) *
    +        //  log(P(w|c) not in class (provided by broadcast))
    +        sumPwcNotFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4))
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +          .join(wordsInText).where(0).equalTo(0) {
    +          (foundW, wordsIT) => (foundW._1, foundW._2, foundW._3, wordsIT._2)
    +        }.map(new RichMapFunction[(Int, String, Int, Int), (Int, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Double] = mutable
    +            .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +              .asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._3)
    +            }
    +          }
    +
    +          override def map(value: (Int, String, Int, Int)): (Int, String, Double) = {
    +            (value._1, value._2, (value._4 - value._3) * broadcastMap(value._2))
    +          }
    +        }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +      } else {
    +        /** if the word frequency is changed (as in SR1 = 1, SR1 = 2, R1 = 1), the
    +          * sumPwcNotFoundWords can not be calculated as above. They must be calculated the same way
    +          * as for the sumPwcFoundWords data set (because each word frequency for each word is
    +          * important).
    +          *
    +          * Prepare foundWords data set for differenz
    +          * 1. Map: discard log(P(w|c) and wordCount from foundWords (id -> class -> word)
    +          * = all words that are known in each document for each class
    +          *
    +          * Prepare wordsAndCounts data set for difference
    +          * 1. FlatMap:create for every tuple a (document id -> class -> word -> word count)
    +          *    tuple for every class
    +          *
    +          * Create notFoundWords data set
    +          * 1: CoGroup: Only tuples (id, class, word) that are not in preparedFoundWords
    +          *
    +          * The result is a data set, that contains all words for each document for each class
    +          * that are not part of that class and the word counts.
    +          *
    +          * Then calcualte sumPwcNotfoundWords
    +          */
    +
    +        val preparedFoundWords: DataSet[(Int, String, String)] = foundWords
    +          .map(line => (line._1, line._2, line._3))
    +
    +        val wordsAndCountsExtended: DataSet[(Int, String, String, Int)] = wordsAndCount
    +          .flatMap(new Extender())
    +          .withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +        val notFoundWords: DataSet[(Int, String, String, Int)] = wordsAndCountsExtended
    +          .coGroup(preparedFoundWords).where(0, 1, 2)
    +          .equalTo(0, 1, 2)(new DifferenceCoGrouper) //(id -> class -> word -> word count)
    +
    +        if ((sr1 == 1 && r1 == 0) || (sr1 == 2 && r1 == 0)) {
    +          //calculate the sum of all Pwc for every not found word (id -> class -> sumPwcNotFound)
    +          // 1. Map: calculate the pwc value for every word (id -> class -> pwc)
    +          // 2. Sum: Sum these pwc values for each class and document
    +          sumPwcNotFoundWords = notFoundWords
    +            .map(new RichMapFunction[(Int, String, String, Int), (Int, String, Double)] {
    +
    +            var broadcastMap: mutable.Map[String, Double] = mutable
    +              .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +            override def open(config: Configuration): Unit = {
    +              val collection = getRuntimeContext
    +                .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +                .asScala
    +              for (record <- collection) {
    +                broadcastMap.put(record._1, record._3)
    +              }
    +            }
    +
    +            override def map(value: (Int, String, String, Int)): (Int, String, Double) = {
    +              if (sr1 == 1 && r1 == 0) {
    +                //same as sr1 == 0, but there is no multiplication with the word counts
    +                return (value._1, value._2, broadcastMap(value._2))
    +              } else if (sr1 == 2 && r1 == 0) {
    +                //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +                return (value._1, value._2, Math.log(value._4 + 1) * broadcastMap(value._2))
    +              }
    +              throw new RuntimeException("sumPwcNotFound could not be calculated because you" +
    +                "choosed a not allowed parameter combination.")
    +            }
    +          }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +            .groupBy(0, 1).sum(2)
    +        } else if ((sr1 == 0 && r1 == 1) || (sr1 == 2 && r1 == 1)) {
    +
    +          //same as r1 = 0, but the word frequency is multiplied with
    +          //log (n_d(c) / n_d(w_t)) for that a join with the improvementData data set must be
    +          //performed first to get the needed additional information.
    +          // 1. Join with improvement data: (id -> class ->  word count ->
    +            // log (number of documents in class / word frequency in all classes))
    +          // 2a. Map: (id -> class -> word count * log (see above) * log(P(w|c))
    +          // 2b. Map: (id -> class -> log(word count + 1) * log (see above) * log(P(w|c))
    +          sumPwcNotFoundWords = notFoundWords
    +            .join(improvementData).where(2).equalTo(0) {
    +            (nf, imp) => (nf._1, nf._2, nf._4, imp._2)
    +          }.map(new RichMapFunction[(Int, String, Int, Double), (Int, String, Double)] {
    +
    +            var broadcastMap: mutable.Map[String, Double] = mutable
    +              .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +            override def open(config: Configuration): Unit = {
    --- End diff --
    
    same as above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964510
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    --- End diff --
    
    Could you rename the test case to something `xxxITSuite` like the other tests?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442749
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    --- End diff --
    
    indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442789
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    --- End diff --
    
    Move the mapping to a separate line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442370
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    --- End diff --
    
    indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by tillrohrmann <gi...@git.apache.org>.
Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r39964588
  
    --- Diff: flink-staging/flink-ml/src/test/scala/org/apache/flink/ml/classification/MultinomialNaiveBayesValidation.scala ---
    @@ -0,0 +1,422 @@
    +/*
    + * 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
    +
    +import java.io.File
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.flink.api.scala._
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.test.util.FlinkTestBase
    +import org.scalatest.{FlatSpec, Matchers}
    +
    +import scala.util.Sorting
    +
    +/**
    + * This test is used to compare whether the different versions of [[MultinomialNaiveBayes]]
    + * resulting of the chooseable "possibilities" give the same result, what they should.
    + */
    +class MultinomialNaiveBayesValidation extends FlatSpec with Matchers with FlinkTestBase {
    +
    +  val outputFolder = "tmp/"
    +
    +
    +  behavior of "The MultinomialNaiveBayes implementation"
    +
    +
    +  it should "train the classifier with the basic configuration" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes()
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "basicConfigModelWord.csv", outputFolder +
    +      "basicConfigModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the basicConfigModel model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +    val nnb = MultinomialNaiveBayes()
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/basicConfigModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder +
    +        "/basicConfigModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "basicConfigSolution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +    env.execute()
    +
    +  }
    +
    +  it should "validate, that basicConfig predicted everything but 357 and 358 correctly" in {
    +    val basicA = scala.io.Source.fromFile(outputFolder + "basicConfigSolution.csv").getLines()
    +      .toArray
    +    for (line <- basicA) {
    +      val split = line.split("\t")
    +      if (split(0).toInt <= 10) {
    +        assert(split(1).equals("business"))
    +      } else if (split(0).toInt <= 356 || split(0).toInt >= 359) {
    +        assert(split(1).equals("entertainment"))
    +      } else if (split(0).toInt == 357 || split(0).toInt == 358) {
    +        assert(split(1).equals("business")) //wrong predicted, but we want this
    +      } else {
    +        fail("unknown identifier number in basicConfigSolution.csv" + split(0))
    +      }
    +    }
    +
    +  }
    +
    +  it should "train the classifier with p1 = 0" in {
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +    val nnb = MultinomialNaiveBayes().setP1(0)
    +
    +    val trainingDS = env.fromCollection(Classification.bbcTrainData)
    +    nnb.fit(trainingDS)
    +
    +    nnb.saveModelDataSet(outputFolder + "p1_0ModelWord.csv", outputFolder + "p1_0ModelClass.csv")
    +
    +    env.execute()
    +  }
    +
    +  it should "use the p1 = 0 model to predict" in {
    +
    +    val env = ExecutionEnvironment.getExecutionEnvironment
    +    env.setParallelism(1)
    +
    +
    +    val nnb = MultinomialNaiveBayes().setP2(0)
    +    nnb.setModelDataSet(env.readCsvFile[(String, String, Double)](outputFolder +
    +      "/p1_0ModelWord.csv", "\n", "|"),
    +      env.readCsvFile[(String, Double, Double)](outputFolder + "/p1_0ModelClass.csv", "\n", "|"))
    +
    +    val solution = nnb.predict(env.fromCollection(Classification.bbcTestData))
    +    solution.writeAsCsv(outputFolder + "p1_0Solution.csv", "\n", "\t", WriteMode.OVERWRITE)
    +
    +
    +    env.execute()
    --- End diff --
    
    verification?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: Pull Request

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-142059639
  
    Hi, I see 5 different types of comments that need to be considered for the new pull request:
    * rename pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes Classification @uce 
    * rename MultinomialNaiveBayesValidation to MultinomialNaiveBayesITSuite @tillrohrmann 
    * result verifications: they are already included (line 158 and line 266 and line 374) @tillrohrmann 
    * remove some line breaks @tillrohrmann 
    * merge commit history to one commit @rmetzger 
    
    Anything else?
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on the pull request:

    https://github.com/apache/flink/pull/1156#issuecomment-143705252
  
    FYI: styling issues fixed, I will now extend my documentation and have a look at the <code>mapWithBcSet</code> method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40444317
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    +                                predictParameters: ParameterMap,
    +                                input: DataSet[(Int, String)]): DataSet[(Int, String)] = {
    +
    +      if (instance.wordRelatedModelData.isEmpty || instance.classRelatedModelData.isEmpty) {
    +        throw new RuntimeException("The NormalNaiveBayes has not been fitted to the " +
    +            "data. This is necessary before a prediction on other data can be made.")
    +      }
    +
    +      val wordRelatedModelData = instance.wordRelatedModelData.get
    +      val classRelatedModelData = instance.classRelatedModelData.get
    +
    +      val resultingParameters = instance.parameters ++ predictParameters
    +
    +      //split the texts from the input data set into its words
    +      val words: DataSet[(Int, String)] = input.flatMap {
    +        pair => pair._2.split(" ").map { word => (pair._1, word)}
    +      }
    +
    +      //genreate word counts for each word with a key
    +      // 1. Map: put a 1 to each key
    +      // 2. Group-Reduce: group by id and word and sum the 1s
    +      val wordsAndCount: DataSet[(Int, String, Int)] = words.map(line => (line._1, line._2, 1))
    +        .groupBy(0, 1).sum(2) // (id -> word -> word count in text)
    +
    +      //calculate the count of all words for a text identified by its key
    +      val wordsInText: DataSet[(Int, Int)] = wordsAndCount.map(line => (line._1, line._3))
    +        .groupBy(0).sum(1) //(id -> all words in text)
    +
    +      //generate a data set containing all words that are in model for each id, class pair
    +      // 1. Join: wordRelatedModelData with wordsAndCount on
    +      //  words (id -> class -> word ->  word count -> log(P(w|c))
    +      val foundWords: DataSet[(Int, String, String, Int, Double)] = wordRelatedModelData
    +        .joinWithHuge(wordsAndCount).where(1).equalTo(1) {
    +        (wordR, wordsAC) => (wordsAC._1, wordR._1, wordsAC._2, wordsAC._3, wordR._3)
    +      }
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +      //RENNIE 1: transform document frequency
    +
    +      val sr1 = resultingParameters(SR1)
    +      val r1 = resultingParameters(R1)
    +      var improvementData: DataSet[(String, Double)] = null
    +
    +      if (r1 == 1) {
    +        //The improvementData data set is needed
    +        if (instance.improvementData.isEmpty) {
    +          throw new RuntimeException("R1 = 1, for that additional data is needed, but it was not" +
    +            "found. Make sure to set R1 = 1 when fitting the training data.")
    +        }
    +        improvementData = instance.improvementData.get
    +      }
    +
    +      if (sr1 == 1 && r1 == 1) {
    +        throw new RuntimeException("Parameter sr1 and r1 are both set to 1, which is not allowed.")
    +      }
    +
    +      var sumPwcFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumpwc for found words
    +        // 1. Map: Remove unneded information from foundWords and calculate the sumP(w|c) for each
    +        //  word (id -> class -> word count * log(P(w|c))
    +        // 2. Group-Reduce: on id and class, sum all (word count * log(P(w|c))) results
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4 * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 1 && r1 == 0) {
    +        //same as sr1 == 0, but there is no multiplication with the word counts
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 2 && r1 == 0) {
    +        //same es sr1 == 0, but multiplication with log(wordcount + 1)
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, Math.log(line._4 + 1) * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 0 && r1 == 1) {
    +        //same as r1 = 0, but the word frequency is multiplied with with log (n_d(c) / n_d(w_t))
    +        //for that a join with the improvementData data set must be performed first to get the
    +        //needed additional information.
    +        // Join: (id -> class ->  word count * log P(w|c) * log (number of documents in class /
    +                                                            // word frequency in all classes)
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, found._4 * found._5 * imp._2)
    +          }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      } else if (sr1 == 2 && r1 == 1) {
    +        //combination of r1 = 1 and sr1 =2
    +        sumPwcFoundWords = foundWords
    +          .joinWithTiny(improvementData).where(2).equalTo(0) {
    +          (found, imp) => (found._1, found._2, Math.log(found._4 + 1) * found._5 * imp._2)
    +        }.groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c)) */
    +      }
    +
    +      var sumPwcNotFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumwpc for words that are not in model in that class
    +        // 1. Map: Discard word and log(P(w|c) from foundWords
    +        // 2. Group-Reduce: calculate sum count of found words for each document,
    +        //  class pair (id -> class -> sum(wordCount))
    +        // 3. Join: with wordsInText on id, to get the count of all words per document
    +        // 4. Map: calculate sumPWcNotFound (id -> class ->
    +        //  (all words in document - found word in document) *
    +        //  log(P(w|c) not in class (provided by broadcast))
    +        sumPwcNotFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4))
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +          .join(wordsInText).where(0).equalTo(0) {
    +          (foundW, wordsIT) => (foundW._1, foundW._2, foundW._3, wordsIT._2)
    +        }.map(new RichMapFunction[(Int, String, Int, Int), (Int, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Double] = mutable
    +            .Map[String, Double]() //class -> log(P(w|c) not found word in class)
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Double, Double)]("classRelatedModelData")
    +              .asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._3)
    +            }
    +          }
    +
    +          override def map(value: (Int, String, Int, Int)): (Int, String, Double) = {
    +            (value._1, value._2, (value._4 - value._3) * broadcastMap(value._2))
    +          }
    +        }).withBroadcastSet(classRelatedModelData, "classRelatedModelData")
    +
    +      } else {
    +        /** if the word frequency is changed (as in SR1 = 1, SR1 = 2, R1 = 1), the
    --- End diff --
    
    `/** */` isn't used inside functions. Use `//` instead.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40442163
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    --- End diff --
    
    As above, perhaps move `null` to the previous line. [I'm not sure about this. Maybe this is just a matter of personal taste. Perhaps someone else will weigh in at some point.]


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40443790
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    +                                predictParameters: ParameterMap,
    +                                input: DataSet[(Int, String)]): DataSet[(Int, String)] = {
    +
    +      if (instance.wordRelatedModelData.isEmpty || instance.classRelatedModelData.isEmpty) {
    +        throw new RuntimeException("The NormalNaiveBayes has not been fitted to the " +
    +            "data. This is necessary before a prediction on other data can be made.")
    +      }
    +
    +      val wordRelatedModelData = instance.wordRelatedModelData.get
    +      val classRelatedModelData = instance.classRelatedModelData.get
    +
    +      val resultingParameters = instance.parameters ++ predictParameters
    +
    +      //split the texts from the input data set into its words
    +      val words: DataSet[(Int, String)] = input.flatMap {
    +        pair => pair._2.split(" ").map { word => (pair._1, word)}
    +      }
    +
    +      //genreate word counts for each word with a key
    +      // 1. Map: put a 1 to each key
    +      // 2. Group-Reduce: group by id and word and sum the 1s
    +      val wordsAndCount: DataSet[(Int, String, Int)] = words.map(line => (line._1, line._2, 1))
    +        .groupBy(0, 1).sum(2) // (id -> word -> word count in text)
    +
    +      //calculate the count of all words for a text identified by its key
    +      val wordsInText: DataSet[(Int, Int)] = wordsAndCount.map(line => (line._1, line._3))
    +        .groupBy(0).sum(1) //(id -> all words in text)
    +
    +      //generate a data set containing all words that are in model for each id, class pair
    +      // 1. Join: wordRelatedModelData with wordsAndCount on
    +      //  words (id -> class -> word ->  word count -> log(P(w|c))
    +      val foundWords: DataSet[(Int, String, String, Int, Double)] = wordRelatedModelData
    +        .joinWithHuge(wordsAndCount).where(1).equalTo(1) {
    +        (wordR, wordsAC) => (wordsAC._1, wordR._1, wordsAC._2, wordsAC._3, wordR._3)
    +      }
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +      //RENNIE 1: transform document frequency
    +
    +      val sr1 = resultingParameters(SR1)
    +      val r1 = resultingParameters(R1)
    +      var improvementData: DataSet[(String, Double)] = null
    +
    +      if (r1 == 1) {
    +        //The improvementData data set is needed
    +        if (instance.improvementData.isEmpty) {
    +          throw new RuntimeException("R1 = 1, for that additional data is needed, but it was not" +
    +            "found. Make sure to set R1 = 1 when fitting the training data.")
    +        }
    +        improvementData = instance.improvementData.get
    +      }
    +
    +      if (sr1 == 1 && r1 == 1) {
    +        throw new RuntimeException("Parameter sr1 and r1 are both set to 1, which is not allowed.")
    +      }
    +
    +      var sumPwcFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumpwc for found words
    +        // 1. Map: Remove unneded information from foundWords and calculate the sumP(w|c) for each
    +        //  word (id -> class -> word count * log(P(w|c))
    +        // 2. Group-Reduce: on id and class, sum all (word count * log(P(w|c))) results
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4 * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    --- End diff --
    
    indentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by JonathanH5 <gi...@git.apache.org>.
Github user JonathanH5 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r41737787
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    --- End diff --
    
    Hi, can you give some hints how I use the mapWithBcSet now? I replaced my package.scala with the one you provided. Can I simply say:
    <code>
    val set2 = set1.mapWithBcSet
    </code>
    
    instead of 
    <code>
    val set2 = set1.map().withBroadcastSet
    </code>
    
    ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request: [FLINK-1719] [ml] Add Multinomial Naive Bayes ...

Posted by sachingoel0101 <gi...@git.apache.org>.
Github user sachingoel0101 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1156#discussion_r40443916
  
    --- Diff: flink-staging/flink-ml/src/main/scala/org/apache/flink/ml/classification/MultinomialNaiveBayes.scala ---
    @@ -0,0 +1,900 @@
    +/*
    + * 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
    +
    +import java.{lang, util}
    +
    +import org.apache.flink.api.common.functions._
    +import org.apache.flink.api.scala._
    +import org.apache.flink.configuration.Configuration
    +import org.apache.flink.core.fs.FileSystem.WriteMode
    +import org.apache.flink.ml.common.{ParameterMap, Parameter}
    +import org.apache.flink.ml.pipeline.{PredictDataSetOperation, FitOperation, Predictor}
    +import org.apache.flink.util.Collector
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable
    +import scala.collection.mutable.ListBuffer
    +import scala.collection.mutable.Map
    +
    +/**
    + * While building the model different approaches need to be compared.
    + * For that purpose the fitParameters are used. Every possibility that might enhance
    + * the implementation can be chosen separately by using the following list of parameters:
    + *
    + * Possibility 1: way of calculating document count
    + *  P1 = 0 -> use .count() to get count of all documents
    + *  P1 = 1 -> use a reducer and a mapper to create a broadcast data set containing the count of
    + *    all documents
    + *
    + * Possibility 2: all words in class (order of operators)
    + *    If p2 = 1 improves the speed, many other calculations must switch their operators, too.
    + *  P2 = 0 -> first the reducer, than the mapper
    + *  P2 = 1 -> first the mapper, than the reducer
    + *
    + * Possibility 3: way of calculating pwc
    + *  P2 = 0 -> join singleWordsInClass and allWordsInClass to wordsInClass data set
    + *  P3 = 1 -> work on singleWordsInClass data set and broadcast allWordsInClass data set
    + *
    + * Schneider/Rennie 1: ignore/reduce word frequency information
    + *  SR1 = 0 -> word frequency information is not ignored
    + *  SR1 = 1 -> word frequency information is ignored (Schneiders approach)
    + *  SR1 = 2 -> word frequency information is reduced (Rennies approach)
    + *
    + * Schneider1: ignore P(c_j) in cMAP formula
    + *  S1 = 0 -> normal cMAP formula
    + *  S2 = 1 -> cMAP without P(c_j)
    + *
    + * Rennie1: transform document frequency
    + *  R1 = 0 -> normal formula
    + *  R1 = 1 -> apply inverse document frequecy
    + * Note: if R1 = 1 and SR1 = 2, both approaches get applied.
    + *
    + */
    +class MultinomialNaiveBayes extends Predictor[MultinomialNaiveBayes] {
    +
    +  import MultinomialNaiveBayes._
    +
    +  //The model, that stores all needed information that are related to one specific word
    +  var wordRelatedModelData: Option[DataSet[(String, String, Double)]] =
    +    None // (class name -> word -> log P(w|c))
    +
    +  //The model, that stores all needed information that are related to one specifc class+
    +  var classRelatedModelData: Option[DataSet[(String, Double, Double)]] =
    +    None // (class name -> p(c) -> log p(w|c) not in class)
    +
    +  //A data set that stores additional needed information for some of the improvements
    +  var improvementData: Option[DataSet[(String, Double)]] =
    +    None // (word -> log number of documents in all classes / word frequency in all classes
    +
    +  // ============================== Parameter configuration ========================================
    +
    +  def setP1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P1, value)
    +    this
    +  }
    +
    +  def setP2(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P2, value)
    +    this
    +  }
    +
    +  def setP3(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(P3, value)
    +    this
    +  }
    +
    +  def setSR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(SR1, value)
    +    this
    +  }
    +
    +  def setS1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(S1, value)
    +    this
    +  }
    +
    +  def setR1(value: Int): MultinomialNaiveBayes = {
    +    parameters.add(R1, value)
    +    this
    +  }
    +
    +  // =============================================== Methods =======================================
    +
    +  /**
    +   * Save already existing model data created by the NaiveBayes algorithm. Requires the designated
    +   * locations. The saved data is a representation of the [[wordRelatedModelData]] and
    +   * [[classRelatedModelData]].
    +   * @param wordRelated, the save location for the wordRelated data
    +   * @param classRelated, the save location for the classRelated data
    +   */
    +  def saveModelDataSet(wordRelated: String, classRelated: String) : Unit = {
    +    wordRelatedModelData.get.writeAsCsv(wordRelated, "\n", "|", WriteMode.OVERWRITE)
    +    classRelatedModelData.get.writeAsCsv(classRelated, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Save the improvment data set. Requires the designated save location. The saved data is a
    +   * representation of the [[improvementData]] data set.
    +   * @param path, the save location for the improvment data
    +   */
    +  def saveImprovementDataSet(path: String) : Unit = {
    +    improvementData.get.writeAsCsv(path, "\n", "|", WriteMode.OVERWRITE)
    +  }
    +
    +  /**
    +   * Sets the [[wordRelatedModelData]] and the [[classRelatedModelData]] to the given data sets.
    +   * @param wordRelated, the data set representing the wordRelated model
    +   * @param classRelated, the data set representing the classRelated model
    +   */
    +  def setModelDataSet(wordRelated : DataSet[(String, String, Double)],
    +                      classRelated: DataSet[(String, Double, Double)]) : Unit = {
    +    this.wordRelatedModelData = Some(wordRelated)
    +    this.classRelatedModelData = Some(classRelated)
    +  }
    +
    +  def setImprovementDataSet(impSet : DataSet[(String, Double)]) : Unit = {
    +    this.improvementData = Some(impSet)
    +  }
    +
    +}
    +
    +object MultinomialNaiveBayes {
    +
    +  // ========================================== Parameters =========================================
    +  case object P1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P2 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object P3 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object SR1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object S1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  case object R1 extends Parameter[Int] {
    +    override val defaultValue: Option[Int] = Some(0)
    +  }
    +
    +  // ======================================== Factory Methods ======================================
    +  def apply(): MultinomialNaiveBayes = {
    +    new MultinomialNaiveBayes()
    +  }
    +
    +  // ====================================== Operations =============================================
    +  /**
    +   * Trains the models to fit the training data. The resulting
    +   * [[MultinomialNaiveBayes.wordRelatedModelData]] and
    +   * [[MultinomialNaiveBayes.classRelatedModelData]] are stored in the [[MultinomialNaiveBayes]]
    +   * instance.
    +   */
    +
    +  implicit val fitNNB = new FitOperation[MultinomialNaiveBayes, (String, String)] {
    +    /**
    +     * The [[FitOperation]] used to create the model. Requires an instance of
    +     * [[MultinomialNaiveBayes]], a [[ParameterMap]] and the input data set. This data set
    +     * maps (string -> string) containing (label -> text, words separated by ",")
    +     * @param instance of [[MultinomialNaiveBayes]]
    +     * @param fitParameters, additional parameters
    +     * @param input, the to processed data set
    +     */
    +    override def fit(instance: MultinomialNaiveBayes,
    +                     fitParameters: ParameterMap,
    +                     input: DataSet[(String, String)]): Unit = {
    +
    +      val resultingParameters = instance.parameters ++ fitParameters
    +
    +      //Count the amount of documents for each class.
    +      // 1. Map: replace the document text by a 1
    +      // 2. Group-Reduce: sum the 1s by class
    +      val documentsPerClass: DataSet[(String, Int)] = input.map { input => (input._1, 1)}
    +        .groupBy(0).sum(1) // (class name -> count of documents)
    +
    +      //Count the amount of occurrences of each word for each class.
    +      // 1. FlatMap: split the document into its words and add a 1 to each tuple
    +      // 2. Group-Reduce: sum the 1s by class, word
    +      var singleWordsInClass: DataSet[(String, String, Int)] = input
    +        .flatMap(new SingleWordSplitter())
    +        .groupBy(0, 1).sum(2) // (class name -> word -> count of that word)
    +
    +      //POSSIBILITY 2: all words in class (order of operators)
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //the allWordsInClass data set does only contain distinct
    +        //words for schneiders approach: ndw(cj), nothing changes for rennies approach
    +
    +      val p2 = resultingParameters(P2)
    +
    +      val sr1 = resultingParameters(SR1)
    +
    +      var allWordsInClass: DataSet[(String, Int)] =
    +        null // (class name -> count of all words in that class)
    +
    +      if (p2 == 0) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Reduce: add the count for each word in a class together
    +          // 2. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass.groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3)) // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          // 3. Map: remove the field that contains the word
    +          allWordsInClass = singleWordsInClass
    +            .map(singleWords => (singleWords._1, singleWords._2, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) =>
    +              (singleWords1._1, singleWords1._2, singleWords1._3 + singleWords2._3)
    +          }.map(singleWords =>
    +            (singleWords._1, singleWords._3))//(class name -> count of distinct words in that class)
    +        }
    +      } else if (p2 == 1) {
    +        if (sr1 == 0 || sr1 == 2) {
    +          //Count all the words for each class.
    +          // 1. Map: remove the field that contains the word
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, singleWords._3))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of all words in that class)
    +        } else if (sr1 == 1) {
    +          //Count all distinct words for each class.
    +          // 1. Map: remove the field that contains the word, set the word count to 1
    +          // 2. Reduce: add the count for each word in a class together
    +          allWordsInClass = singleWordsInClass.map(singleWords => (singleWords._1, 1))
    +            .groupBy(0).reduce {
    +            (singleWords1, singleWords2) => (singleWords1._1, singleWords1._2 + singleWords2._2)
    +          } // (class name -> count of distinct words in that class)
    +        }
    +
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +      //END POSSIBILITY 2
    +
    +      //POSSIBILITY 1: way of calculating document count
    +      val p1 = resultingParameters(P1)
    +
    +      var pc: DataSet[(String, Double)] = null // (class name -> P(c) in class)
    +
    +      if (p1 == 0) {
    +        val documentsCount: Double = input.count() //count of all documents
    +        //Calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        pc = documentsPerClass.map(line => (line._1, line._2 / documentsCount))
    +
    +      } else if (p1 == 1) {
    +        //Create a data set that contains only one double value: the count of all documents
    +        // 1. Reduce: At the count of documents together
    +        // 2. Map: Remove field that contains document identifier
    +        val documentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) //(count of all documents)
    +
    +        //calculate P(c)
    +        // 1. Map: divide count of documents for a class through total count of documents
    +        //    (only element in documentCount data set)
    +        pc = documentsPerClass.map(new RichMapFunction[(String, Int), (String, Double)] {
    +
    +            var broadcastSet: util.List[Double] = null
    +
    +            override def open(config: Configuration): Unit = {
    +              broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("documentCount")
    +              if (broadcastSet.size() != 1) {
    +                throw new RuntimeException("The document count data set used by p1 = 1 has the " +
    +                  "wrong size! Please use p1 = 0 if the problem can not be solved.")
    +              }
    +            }
    +
    +            override def map(value: (String, Int)): (String, Double) = {
    +              (value._1, value._2 / broadcastSet.get(0))
    +            }
    +          }).withBroadcastSet(documentCount, "documentCount")
    +      }
    +      //END POSSIBILITY 1
    +
    +      // (list of all words, but distinct)
    +      val vocabulary = singleWordsInClass.map(tuple => (tuple._2, 1)).distinct(0)
    +      // (count of items in vocabulary list)
    +      val vocabularyCount: Double = vocabulary.count()
    +
    +      //calculate the P(w|c) value for words, that are not part of a class, needed for smoothing
    +      // 1. Map: use P(w|c) formula with smoothing with n(c_j, w_t) = 0
    +      val pwcNotInClass: DataSet[(String, Double)] = allWordsInClass
    +        .map(line =>
    +          (line._1, 1 / (line._2 + vocabularyCount))) // (class name -> P(w|c) word not in class)
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +        //The singleWordsInClass data set must be changed before, the calculation of pwc starts for
    +        //schneider, it needs this form classname -> word -> number of documents containing wt in cj
    +
    +      if (sr1 == 1) {
    +        //Calculate the required data set (see above)
    +        // 1. FlatMap: class -> word -> 1 (one tuple for each document in which this word occurs)
    +        // 2. Group-Reduce: sum all 1s where the first two fields equal
    +        // 3. Map: Remove unesseccary count of word and replace with 1
    +        singleWordsInClass = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .groupBy(0, 1)
    +          .reduce((line1, line2) => (line1._1, line1._2, line1._3 + line2._3))
    +      }
    +
    +      //END SCHNEIDER/RENNIE 1
    +
    +      //POSSIBILITY 3: way of calculating pwc
    +
    +      val p3 = resultingParameters(P3)
    +
    +      var pwc: DataSet[(String, String, Double)] = null // (class name -> word -> P(w|c))
    +
    +      if (p3 == 0) {
    +
    +          //Join the singleWordsInClass data set with the allWordsInClass data set to use the
    +          //information for the calculation of p(w|c).
    +          val wordsInClass = singleWordsInClass
    +            .join(allWordsInClass).where(0).equalTo(0) {
    +            (single, all) => (single._1, single._2, single._3, all._2)
    +          } // (class name -> word -> count of that word -> count of all words in that class)
    +
    +          //calculate the P(w|c) value for each word in each class
    +          // 1. Map: use normal P(w|c) formula
    +          pwc = wordsInClass.map(line => (line._1, line._2, (line._3 + 1) /
    +            (line._4 + vocabularyCount)))
    +
    +      } else if (p3 == 1) {
    +
    +        //calculate the P(w|c) value for each word in class
    +        //  1. Map: use normal P(w|c) formula / use the
    +        pwc = singleWordsInClass.map(new RichMapFunction[(String, String, Int),
    +          (String, String, Double)] {
    +
    +          var broadcastMap: mutable.Map[String, Int] = mutable.Map[String, Int]()
    +
    +
    +          override def open(config: Configuration): Unit = {
    +            val collection = getRuntimeContext
    +              .getBroadcastVariable[(String, Int)]("allWordsInClass").asScala
    +            for (record <- collection) {
    +              broadcastMap.put(record._1, record._2)
    +            }
    +          }
    +
    +          override def map(value: (String, String, Int)): (String, String, Double) = {
    +            (value._1, value._2, (value._3 + 1) / (broadcastMap(value._1) + vocabularyCount))
    +          }
    +        }).withBroadcastSet(allWordsInClass, "allWordsInClass")
    +
    +      }
    +
    +      //END POSSIBILITY 3
    +
    +      //stores all the word related information in one data set
    +      // 1. Map: Caluclate logarithms
    +      val wordRelatedModelData = pwc.map(line => (line._1, line._2, Math.log(line._3)))
    +
    +      //store all class related information in one data set
    +      // 1. Join: P(c) data set and P(w|c) data set not in class and calculate logarithms
    +      val classRelatedModelData = pc.join(pwcNotInClass)
    +        .where(0).equalTo(0) {
    +        (line1, line2) => (line1._1, Math.log(line1._2), Math.log(line2._2))
    +      } // (class name -> log(P(c)) -> log(P(w|c) not in class))
    +
    +      instance.wordRelatedModelData = Some(wordRelatedModelData)
    +      instance.classRelatedModelData = Some(classRelatedModelData)
    +
    +      //RENNIE 1: transform document frequency
    +        //for this, the improvementData set must be set
    +        //calculate (word -> log number of documents in all classes / docs with that word)
    +
    +      val r1 = resultingParameters(R1)
    +
    +      if (r1 == 1) {
    +        val totalDocumentCount: DataSet[(Double)] = documentsPerClass
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +          .map(line => line._2) // (count of all documents)
    +
    +        //number of occurences over all documents of all classes
    +        val wordCountTotal = input
    +          .flatMap(new SingleDistinctWordSplitter())
    +          .map(line => (line._2, 1))
    +          .groupBy(0)
    +          .reduce((line1, line2) => (line1._1, line1._2 + line2._2))
    +           // (word -> count of documents with that word)
    +
    +        val improvementData = wordCountTotal.map(new RichMapFunction[(String, Int),
    +          (String, Double)] {
    +
    +          var broadcastSet: util.List[Double] = null
    +
    +          override def open(config: Configuration): Unit = {
    +            broadcastSet = getRuntimeContext.getBroadcastVariable[Double]("totalDocumentCount")
    +            if (broadcastSet.size() != 1) {
    +              throw new RuntimeException("The total document count data set used by 11 = 1 has " +
    +                "the wrong size! Please use r1 = 0 if the problem can not be solved.")
    +            }
    +          }
    +
    +          override def map(value: (String, Int)): (String, Double) = {
    +            (value._1, Math.log(broadcastSet.get(0) / value._2))
    +          }
    +        }).withBroadcastSet(totalDocumentCount, "totalDocumentCount")
    +
    +        instance.improvementData = Some(improvementData)
    +      }
    +
    +    }
    +  }
    +
    +  // Model (String, String, Double, Double, Double)
    +  implicit def predictNNB = new PredictDataSetOperation[
    +    MultinomialNaiveBayes,
    +    (Int, String),
    +    (Int, String)]() {
    +
    +    override def predictDataSet(instance: MultinomialNaiveBayes,
    +                                predictParameters: ParameterMap,
    +                                input: DataSet[(Int, String)]): DataSet[(Int, String)] = {
    +
    +      if (instance.wordRelatedModelData.isEmpty || instance.classRelatedModelData.isEmpty) {
    +        throw new RuntimeException("The NormalNaiveBayes has not been fitted to the " +
    +            "data. This is necessary before a prediction on other data can be made.")
    +      }
    +
    +      val wordRelatedModelData = instance.wordRelatedModelData.get
    +      val classRelatedModelData = instance.classRelatedModelData.get
    +
    +      val resultingParameters = instance.parameters ++ predictParameters
    +
    +      //split the texts from the input data set into its words
    +      val words: DataSet[(Int, String)] = input.flatMap {
    +        pair => pair._2.split(" ").map { word => (pair._1, word)}
    +      }
    +
    +      //genreate word counts for each word with a key
    +      // 1. Map: put a 1 to each key
    +      // 2. Group-Reduce: group by id and word and sum the 1s
    +      val wordsAndCount: DataSet[(Int, String, Int)] = words.map(line => (line._1, line._2, 1))
    +        .groupBy(0, 1).sum(2) // (id -> word -> word count in text)
    +
    +      //calculate the count of all words for a text identified by its key
    +      val wordsInText: DataSet[(Int, Int)] = wordsAndCount.map(line => (line._1, line._3))
    +        .groupBy(0).sum(1) //(id -> all words in text)
    +
    +      //generate a data set containing all words that are in model for each id, class pair
    +      // 1. Join: wordRelatedModelData with wordsAndCount on
    +      //  words (id -> class -> word ->  word count -> log(P(w|c))
    +      val foundWords: DataSet[(Int, String, String, Int, Double)] = wordRelatedModelData
    +        .joinWithHuge(wordsAndCount).where(1).equalTo(1) {
    +        (wordR, wordsAC) => (wordsAC._1, wordR._1, wordsAC._2, wordsAC._3, wordR._3)
    +      }
    +
    +      //SCHNEIDER/RENNIE 1: ignore/reduce word frequency information
    +      //RENNIE 1: transform document frequency
    +
    +      val sr1 = resultingParameters(SR1)
    +      val r1 = resultingParameters(R1)
    +      var improvementData: DataSet[(String, Double)] = null
    +
    +      if (r1 == 1) {
    +        //The improvementData data set is needed
    +        if (instance.improvementData.isEmpty) {
    +          throw new RuntimeException("R1 = 1, for that additional data is needed, but it was not" +
    +            "found. Make sure to set R1 = 1 when fitting the training data.")
    +        }
    +        improvementData = instance.improvementData.get
    +      }
    +
    +      if (sr1 == 1 && r1 == 1) {
    +        throw new RuntimeException("Parameter sr1 and r1 are both set to 1, which is not allowed.")
    +      }
    +
    +      var sumPwcFoundWords: DataSet[(Int, String, Double)] = null
    +
    +      if (sr1 == 0 && r1 == 0) {
    +        //calculate sumpwc for found words
    +        // 1. Map: Remove unneded information from foundWords and calculate the sumP(w|c) for each
    +        //  word (id -> class -> word count * log(P(w|c))
    +        // 2. Group-Reduce: on id and class, sum all (word count * log(P(w|c))) results
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._4 * line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    +          (line1._1, line1._2, line1._3 + line2._3)) //(id -> class -> sum(log(P(w|c))
    +      } else if (sr1 == 1 && r1 == 0) {
    +        //same as sr1 == 0, but there is no multiplication with the word counts
    +        sumPwcFoundWords = foundWords
    +          .map(line => (line._1, line._2, line._5))
    +          .groupBy(0, 1).reduce((line1, line2) =>
    --- End diff --
    
    as above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---