[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-06 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r149215757
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
+   * Default: 15
+   * @group param
+   */
+  final val numNegativeSamples = new IntParam(this, "numNegativeSamples", 
"Number of negative" +
+" samples to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(numNegativeSamples -> 15)
--- End diff --

15 gives better performance than 5. I agree with your point below as well. 
We should make the default parameters ones that result in best performance. 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148968172
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
+   * Default: 15
+   * @group param
+   */
+  final val numNegativeSamples = new IntParam(this, "numNegativeSamples", 
"Number of negative" +
+" samples to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(numNegativeSamples -> 15)
+
+  /** @group getParam */
+  def getNumNegativeSamples: Int = $(numNegativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
+
+  /** @group getParam */
+  def getUnigramTableSize: Int = $(unigramTableSize)
+
+  /**
+   * Sample threshold (parameter t https://arxiv.org/pdf/1310.4546.pdf";>here)
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 0.0
+   * @group param
+   */
+  final val samplingThreshold = new DoubleParam(this, "samplingThreshold", 
"Sampling threshold" +
--- End diff --

perhaps we can rename this to a more precise name, E.g. 
downSamplingTFThreshold or highTFThreshold


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148968255
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
+   * Default: 15
+   * @group param
+   */
+  final val numNegativeSamples = new IntParam(this, "numNegativeSamples", 
"Number of negative" +
+" samples to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(numNegativeSamples -> 15)
+
+  /** @group getParam */
+  def getNumNegativeSamples: Int = $(numNegativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
+
+  /** @group getParam */
+  def getUnigramTableSize: Int = $(unigramTableSize)
+
+  /**
+   * Sample threshold (parameter t https://arxiv.org/pdf/1310.4546.pdf";>here)
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 0.0
+   * @group param
+   */
+  final val samplingThreshold = new DoubleParam(this, "samplingThreshold", 
"Sampling threshold" +
+" to reduce words with high frequencies", ParamValidators.gtEq(0.0))
+  setDefault(samplingThreshold -> 0.0)
--- End diff --

IMO we can use 1e-5 here. Default param should target for best algorithm 
outcome.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148967559
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
+   * Default: 15
+   * @group param
+   */
+  final val numNegativeSamples = new IntParam(this, "numNegativeSamples", 
"Number of negative" +
+" samples to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(numNegativeSamples -> 15)
+
+  /** @group getParam */
+  def getNumNegativeSamples: Int = $(numNegativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
+
+  /** @group getParam */
+  def getUnigramTableSize: Int = $(unigramTableSize)
+
+  /**
+   * Sample threshold (parameter t https://arxiv.org/pdf/1310.4546.pdf";>here)
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 0.0
+   * @group param
+   */
+  final val samplingThreshold = new DoubleParam(this, "samplingThreshold", 
"Sampling threshold" +
+" to reduce words with high frequencies", ParamValidators.gtEq(0.0))
+  setDefault(samplingThreshold -> 0.0)
+
+  /** @group getParam */
+  def getSamplingThreshold: Double = $(samplingThreshold)
+
+  /**
+   * Solver used for Word2Vec.
+   * Supported options are "sg-hs" and "cbow-ns"
--- End diff --

add more details:
sg-hs: Skip-gram with hierarchical softmax
cbow: ...


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148967956
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
+   * Default: 15
+   * @group param
+   */
+  final val numNegativeSamples = new IntParam(this, "numNegativeSamples", 
"Number of negative" +
+" samples to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(numNegativeSamples -> 15)
+
+  /** @group getParam */
+  def getNumNegativeSamples: Int = $(numNegativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
--- End diff --

maybe move the setDefault after setter in Word2Vec, which seems to be a 
recommended practice right now. (Just as the solver)



---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148967766
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
--- End diff --

Rather than exclusion, I would recommend to use precise description here 
and in other places.
The parameter is only used for continuous bag-of-words with negative 
sampling algorithm, i.e. solver = "cbow-ns".


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148967853
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
+   * Default: 15
+   * @group param
+   */
+  final val numNegativeSamples = new IntParam(this, "numNegativeSamples", 
"Number of negative" +
+" samples to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(numNegativeSamples -> 15)
+
+  /** @group getParam */
+  def getNumNegativeSamples: Int = $(numNegativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
--- End diff --

expert param?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148968530
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/impl/Word2VecCBOWSolver.scala 
---
@@ -0,0 +1,371 @@
+/*
+ * 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.spark.ml.feature.impl
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.ml.feature.Word2Vec
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+private [feature] object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
--- End diff --

IMO we should expose batchSize.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-11-05 Thread hhbyyh
Github user hhbyyh commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r148966706
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -105,6 +106,56 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram-Hierachical Softmax based 
estimation.
+   * Default: 15
+   * @group param
+   */
+  final val numNegativeSamples = new IntParam(this, "numNegativeSamples", 
"Number of negative" +
+" samples to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(numNegativeSamples -> 15)
--- End diff --

In other implementation, the default value is 5. 
Is 15 from the original C-implementation?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-11 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r144124146
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143573574
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143572643
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143572667
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143572179
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143572149
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143571594
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143570173
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143570164
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_,

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143569334
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
--- End diff --

There are a few places where the implementation seems counter-intuitive, 
and doesn't directly relate to the papers its based on. I thought it might help 
if someone wants to cross-check the implementation with the C implementation. 
Other than that, I do agree with you, and have tried to name things 
appropriately, other than these key variables around which the implementation 
is based.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143569449
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
--- End diff --

It's not needed here, is it?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143567888
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
--- End diff --

Done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143567788
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
--- End diff --

Done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143567807
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
--- End diff --

Done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143566804
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
--- End diff --

Done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143529694
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala ---
@@ -245,5 +508,28 @@ class Word2VecSuite extends SparkFunSuite with 
MLlibTestSparkContext with Defaul
 model.transform(ngramDF).collect()
   }
 
+  test("Word2Vec works with input that is non-nullable (NGram) - CBOW") {
--- End diff --

Yes. Taking another pass on the tests to remove code duplication.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143529286
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala ---
@@ -189,6 +305,136 @@ class Word2VecSuite extends SparkFunSuite with 
MLlibTestSparkContext with Defaul
 assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 
1E-5)
   }
 
+  test("window size - CBOW") {
+
+val spark = this.spark
+import spark.implicits._
+
+val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10
+val doc = sc.parallelize(Seq(sentence, sentence)).map(line => 
line.split(" "))
+val docDF = doc.zip(doc).toDF("text", "alsotext")
+
+val model = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(5)
+  .setVectorSize(3)
+  .setWindowSize(2)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .fit(docDF)
+
+val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+
+// Increase the window size
+val biggerModel = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(5)
+  .setVectorSize(3)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .setWindowSize(10)
+  .fit(docDF)
+
+val (synonymsLarger, similarityLarger) = model.findSynonyms("a", 
6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+// The similarity score should be very different with the larger window
+assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 
1E-5)
+  }
+
+  test("negative sampling - CBOW") {
+
+val spark = this.spark
+import spark.implicits._
+
+val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10
+val doc = sc.parallelize(Seq(sentence, sentence)).map(line => 
line.split(" "))
+val docDF = doc.zip(doc).toDF("text", "alsotext")
+
+val model = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(2)
+  .setVectorSize(3)
+  .setWindowSize(2)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .fit(docDF)
+
+val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+
+// Increase the window size
--- End diff --

Corrected. Thanks.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143528339
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
+  def setSolver(value: String): this.type = {
+require(solvers.contains(value),
+  s"Solver $value was not supported. Supported options: 
${solvers.mkString(", ")}")
+set(solver, value)
+  }
+  setDefault(solver -> "sg-hs")
+
+  /** @group setParam */
+  @Since("2.2.0")
+  def setNegativeSamples(value: Int): this.type = set(negativeSamples, 
value)
+
+  /** @group setParam */
+  @Since("2.2.0")
+  def setMaxUnigramTableSize(value: Int): this.type = 
set(unigramTableSize, value)
--- End diff --

Done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143528173
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
--- End diff --

Got it. I will have to rebase to master for this change, HasSolver trait 
has been updated since I implemented this. Keeping this for the end.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143516772
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
+"to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(negativeSamples -> 15)
+
+  /** @group getParam */
+  def getNegativeSamples: Int = $(negativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
+
+  /** @group getParam */
+  def getUnigramTableSize: Int = $(unigramTableSize)
+
+  /**
+   * Sample threshold (parameter t https://arxiv.org/pdf/1310.4546.pdf";>here)
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 0.0
+   * @group param
+   */
+  final val sample = new DoubleParam(this, "samplingThreshold", "Sampling 
threshold to reduce " +
--- End diff --

Corrected. Thanks!


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143516595
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
+"to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(negativeSamples -> 15)
+
+  /** @group getParam */
+  def getNegativeSamples: Int = $(negativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
+
+  /** @group getParam */
+  def getUnigramTableSize: Int = $(unigramTableSize)
+
+  /**
+   * Sample threshold (parameter t https://arxiv.org/pdf/1310.4546.pdf";>here)
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 0.0
+   * @group param
+   */
+  final val sample = new DoubleParam(this, "samplingThreshold", "Sampling 
threshold to reduce " +
--- End diff --

Done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143516496
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
--- End diff --

Done


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-09 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143516384
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
--- End diff --

`negativeSamples` is only used when Negative Sampling is used with CBOW 
(or, as in #18123, SkipGram)
SkipGram with Hierarchical Softmax, which was the initial version, is not 
affected by this parameter.

In principle, we now have 3 possible techniques that can be used (another 
one implemented in #18123). Open to suggestions possible values for `solver`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143048261
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143063348
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143051494
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143011936
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
--- End diff --

parentheses


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143020551
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
--- End diff --

`numNegativeSamples`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143029928
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143029396
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143036042
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143051749
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143050445
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
--- End diff --

Yes, I think it does. An expert param maybe.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143050632
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
--- End diff --

"using level 1 and level 2 BLAS"


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143053337
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143034710
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143053094
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143050902
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143020469
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143033624
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143021666
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143051105
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143050245
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143034368
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143034586
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143011531
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
--- End diff --

`sampleTable.length`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143048444
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143050140
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143052196
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143020756
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
--- End diff --

`samplingThreshold`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143048339
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143030468
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
--- End diff --

For random forest, we made a separate `impl` package and put this type of 
code there. I think that's probably a good way to do it here as well. Also, 
this should be a `private[feature]` object.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143007246
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142992652
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143022987
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143053455
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143051704
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143020307
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
--- End diff --

Still looking through algo details, but I find many of the variable names 
unhelpful, these included. What is syn0, syn1? I know what they represent, syn0 
being the word embeddings, not sure what "syn" is. I know much of the code is 
meant to mirror the C implementation, but I'd prefer variable names that are 
clear, rather than a one-to-one mapping


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r143033876
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
+
+  // power to raise the unigram distribution with
+  private val power = 0.75
+
+  private val MAX_EXP = 6
+
+  case class Vocabulary(
+totalWordCount: Long,
+vocabMap: Map[String, Int],
+unigramTable: Array[Int],
+samplingTable: Array[Float])
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * We divide input data into N equally sized random partitions.
+   * We then generate initial weights and broadcast them to the N 
partitions. This way
+   * all the partitions start with the same initial weights. We then run N 
independent
+   * estimations that each estimate a model on a partition. The weights 
learned
+   * from each of the N models are averaged and rebroadcast the weights.
+   * This process is repeated `maxIter` number of times.
+   *
+   * @param input A RDD of strings. Each string would be considered a 
sentence.
+   * @return Estimated word2vec model
+   */
+  def fitCBOW[S <: Iterable[String]](
+  word2Vec: Word2Vec,
+  input: RDD[S]): feature.Word2VecModel = {
+
+val negativeSamples = word2Vec.getNegativeSamples
+val sample = word2Vec.getSample
+
+val Vocabulary(totalWordCount, vocabMap, uniTable, sampleTable) =
+  generateVocab(input, word2Vec.getMinCount, sample, 
word2Vec.getUnigramTableSize)
+val vocabSize = vocabMap.size
+
+assert(negativeSamples < vocabSize, s"Vocab size ($vocabSize) cannot 
be smaller" +
+  s" than negative samples($negativeSamples)")
+
+val seed = word2Vec.getSeed
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = word2Vec.getVectorSize
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapBroadcast = sc.broadcast(vocabMap)
+val unigramTableBroadcast = sc.broadcast(uniTable)
+val sampleTableBroadcast = sc.broadcast(sampleTable)
+
+val windowSize = word2Vec.getWindowSize
+val maxSentenceLength = word2Vec.getMaxSentenceLength
+val numPartitions = word2Vec.getNumPartitions
+
+val digitSentences = input.flatMap { sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapBroadcast.value.get)
+  wordIndexes.grouped(maxSentenceLength).map(_.toArray)
+}.repartition(numPartitions).cache()
+
+val learningRate = word2Vec.getStepSize
+
+val wordsPerPartition = totalWordCount / numPartitions
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+val maxIter = word2Vec.getMaxIter
+for {iteration <- 1 to maxIter} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex { case (i_, 
iter)

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142991123
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
--- End diff --

Yeah, for reference you can just look at how linear regression does the 
`supportedSolvers`. Also, the require isn't necessary, you can just use 
`ParamValidators.inArray[String](supportedSolvers))`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-05 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142990145
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
--- End diff --

"skipgram-hierarchical softmax"


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142218654
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala ---
@@ -245,5 +508,28 @@ class Word2VecSuite extends SparkFunSuite with 
MLlibTestSparkContext with Defaul
 model.transform(ngramDF).collect()
   }
 
+  test("Word2Vec works with input that is non-nullable (NGram) - CBOW") {
--- End diff --

Can this test not just be folded into the one above?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142216498
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/Word2VecCBOWSolver.scala ---
@@ -0,0 +1,344 @@
+/*
+ * 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.spark.ml.feature
+
+import com.github.fommil.netlib.BLAS.{getInstance => blas}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.mllib.feature
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+
+object Word2VecCBOWSolver extends Logging {
+  // learning rate is updated for every batch of size batchSize
+  private val batchSize = 1
--- End diff --

Does it make sense to make this configurable?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142218010
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala ---
@@ -189,6 +305,136 @@ class Word2VecSuite extends SparkFunSuite with 
MLlibTestSparkContext with Defaul
 assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 
1E-5)
   }
 
+  test("window size - CBOW") {
+
+val spark = this.spark
+import spark.implicits._
+
+val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10
+val doc = sc.parallelize(Seq(sentence, sentence)).map(line => 
line.split(" "))
+val docDF = doc.zip(doc).toDF("text", "alsotext")
+
+val model = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(5)
+  .setVectorSize(3)
+  .setWindowSize(2)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .fit(docDF)
+
+val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+
+// Increase the window size
+val biggerModel = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(5)
+  .setVectorSize(3)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .setWindowSize(10)
+  .fit(docDF)
+
+val (synonymsLarger, similarityLarger) = model.findSynonyms("a", 
6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+// The similarity score should be very different with the larger window
+assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 
1E-5)
+  }
+
+  test("negative sampling - CBOW") {
+
+val spark = this.spark
+import spark.implicits._
+
+val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10
+val doc = sc.parallelize(Seq(sentence, sentence)).map(line => 
line.split(" "))
+val docDF = doc.zip(doc).toDF("text", "alsotext")
+
+val model = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(2)
+  .setVectorSize(3)
+  .setWindowSize(2)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .fit(docDF)
+
+val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+
+// Increase the window size
--- End diff --

comment is misleading here 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142215490
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
+"to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(negativeSamples -> 15)
+
+  /** @group getParam */
+  def getNegativeSamples: Int = $(negativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
+
+  /** @group getParam */
+  def getUnigramTableSize: Int = $(unigramTableSize)
+
+  /**
+   * Sample threshold (parameter t https://arxiv.org/pdf/1310.4546.pdf";>here)
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 0.0
+   * @group param
+   */
+  final val sample = new DoubleParam(this, "samplingThreshold", "Sampling 
threshold to reduce " +
--- End diff --

The param doc says "... to reduce with high frequencies". Reduce what? (I 
assume this should be "terms" or "words")


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142216313
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
--- End diff --

Also, minor but I'd prefer "skipgram" (it's short enough for this purpose 
and is more descriptive).


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142221460
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
+  def setSolver(value: String): this.type = {
+require(solvers.contains(value),
+  s"Solver $value was not supported. Supported options: 
${solvers.mkString(", ")}")
+set(solver, value)
+  }
+  setDefault(solver -> "sg-hs")
+
+  /** @group setParam */
+  @Since("2.2.0")
+  def setNegativeSamples(value: Int): this.type = set(negativeSamples, 
value)
+
+  /** @group setParam */
+  @Since("2.2.0")
+  def setMaxUnigramTableSize(value: Int): this.type = 
set(unigramTableSize, value)
--- End diff --

setters should match the params - this should be `setUnigramTableSize`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142215089
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
+"to use with CBOW estimation", ParamValidators.gt(0))
+  setDefault(negativeSamples -> 15)
+
+  /** @group getParam */
+  def getNegativeSamples: Int = $(negativeSamples)
+
+  /**
+   * Unigram table size. The unigram table is used to generate negative 
samples.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 100 million
+   * @group param
+   */
+  final val unigramTableSize = new IntParam(this, "unigramTableSize", "Max 
table size to " +
+"use for generating negative samples", ParamValidators.gt(1))
+  setDefault(unigramTableSize -> 100*1000*1000)
+
+  /** @group getParam */
+  def getUnigramTableSize: Int = $(unigramTableSize)
+
+  /**
+   * Sample threshold (parameter t https://arxiv.org/pdf/1310.4546.pdf";>here)
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 0.0
+   * @group param
+   */
+  final val sample = new DoubleParam(this, "samplingThreshold", "Sampling 
threshold to reduce " +
--- End diff --

Let's be consistent and use `samplingThreshold` for the val name and param 
name.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142218319
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala ---
@@ -189,6 +305,136 @@ class Word2VecSuite extends SparkFunSuite with 
MLlibTestSparkContext with Defaul
 assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 
1E-5)
   }
 
+  test("window size - CBOW") {
+
+val spark = this.spark
+import spark.implicits._
+
+val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10
+val doc = sc.parallelize(Seq(sentence, sentence)).map(line => 
line.split(" "))
+val docDF = doc.zip(doc).toDF("text", "alsotext")
+
+val model = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(5)
+  .setVectorSize(3)
+  .setWindowSize(2)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .fit(docDF)
+
+val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+
+// Increase the window size
+val biggerModel = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(5)
+  .setVectorSize(3)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .setWindowSize(10)
+  .fit(docDF)
+
+val (synonymsLarger, similarityLarger) = model.findSynonyms("a", 
6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+// The similarity score should be very different with the larger window
+assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 
1E-5)
+  }
+
+  test("negative sampling - CBOW") {
+
+val spark = this.spark
+import spark.implicits._
+
+val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10
+val doc = sc.parallelize(Seq(sentence, sentence)).map(line => 
line.split(" "))
+val docDF = doc.zip(doc).toDF("text", "alsotext")
+
+val model = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(2)
+  .setVectorSize(3)
+  .setWindowSize(2)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .fit(docDF)
+
+val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+
+// Increase the window size
+val biggerModel = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(5)
+  .setVectorSize(3)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .setWindowSize(2)
+  .fit(docDF)
+
+val (synonymsLarger, similarityLarger) = model.findSynonyms("a", 
6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+// The similarity score should be very different with the higher 
negative sampling
+assert(math.abs(similarity(5) - similarityLarger(5) / similarity(5)) > 
1E-5)
+  }
+
+  test("sub sampling - CBOW") {
+
+val spark = this.spark
+import spark.implicits._
+
+val sentence = "a q s t q s t b b b s t m s t m q " * 100 + "a c " * 10
+val doc = sc.parallelize(Seq(sentence, sentence)).map(line => 
line.split(" "))
+val docDF = doc.zip(doc).toDF("text", "alsotext")
+
+val model = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(2)
+  .setVectorSize(3)
+  .setWindowSize(2)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .fit(docDF)
+
+val (synonyms, similarity) = model.findSynonyms("a", 6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+
+// Set sub sampling
+val biggerModel = new Word2Vec()
+  .setSolver("cbow-ns")
+  .setMaxUnigramTableSize(1)
+  .setNegativeSamples(2)
+  .setVectorSize(3)
+  .setInputCol("text")
+  .setOutputCol("result")
+  .setSeed(42L)
+  .setWindowSize(2)
+  .setSample(0.01)
+  .fit(docDF)
+
+val (synonymsLarger, similarityLarger) = model.findSynonyms("a", 
6).rdd.map {
+  case Row(w: String, sim: Double) => (w, sim)
+}.collect().unzip
+// The similarity score shou

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142216094
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
--- End diff --

Why "sg-hs"? 


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142220283
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
--- End diff --

In relation to #18123 I wonder if it is necessary to have 3 possible values 
for `solver`? For `skipgram` if we set this param `negativeSamples` to `0` will 
that give the same effect as normal skipgram?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142221266
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -106,6 +106,45 @@ private[feature] trait Word2VecBase extends Params
   /** @group getParam */
   def getMaxSentenceLength: Int = $(maxSentenceLength)
 
+  /**
+   * Number of negative samples to use with CBOW based estimation.
+   * This parameter is ignored for SkipGram based estimation.
+   * Default: 15
+   * @group param
+   */
+  final val negativeSamples = new IntParam(this, "negativeSamples", 
"Number of negative samples " +
--- End diff --

Perhaps we should call it `numNegativeSamples`


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142219791
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala ---
@@ -76,6 +76,49 @@ class Word2VecSuite extends SparkFunSuite with 
MLlibTestSparkContext with Defaul
 }
   }
 
+  test("Word2Vec-CBOW") {
--- End diff --

I feel like there is a lot of duplication in the Word2Vec tests now. It 
would be ok to do as a follow up, but I think things can be cleaned up a lot 
here.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-10-02 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r142215925
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -171,20 +210,46 @@ final class Word2Vec @Since("1.4.0") (
   @Since("2.0.0")
   def setMaxSentenceLength(value: Int): this.type = set(maxSentenceLength, 
value)
 
+  /** @group setParam */
+  @Since("2.2.0")
+  val solvers = Set("sg-hs", "cbow-ns")
--- End diff --

We typically put these types of variables in the companion objects. I don't 
think it's necessary to make it public either.

Also, it may be a good idea to extract the strings in constants.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-05 Thread shubhamchopra
Github user shubhamchopra commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r115009247
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -36,7 +36,10 @@ import org.apache.spark.util.{Utils, VersionUtils}
  * Params for [[Word2Vec]] and [[Word2VecModel]].
  */
 private[feature] trait Word2VecBase extends Params
-  with HasInputCol with HasOutputCol with HasMaxIter with HasStepSize with 
HasSeed {
+  with HasInputCol with HasOutputCol with HasMaxIter with HasStepSize with 
HasSeed with HasSolver {
+  // We currently support SkipGram with Hierarchical Softmax and
+  // Continuous Bag of Words with Negative Sampling
+  private val supportedModels = Array("sg-hs", "cbow-ns")
--- End diff --

Good catch. It is not used. Fixing it.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-04 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114929436
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -36,7 +36,10 @@ import org.apache.spark.util.{Utils, VersionUtils}
  * Params for [[Word2Vec]] and [[Word2VecModel]].
  */
 private[feature] trait Word2VecBase extends Params
-  with HasInputCol with HasOutputCol with HasMaxIter with HasStepSize with 
HasSeed {
+  with HasInputCol with HasOutputCol with HasMaxIter with HasStepSize with 
HasSeed with HasSolver {
+  // We currently support SkipGram with Hierarchical Softmax and
+  // Continuous Bag of Words with Negative Sampling
+  private val supportedModels = Array("sg-hs", "cbow-ns")
--- End diff --

how is this used?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114529585
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
+}.repartition($(numPartitions)).cache()
+
+val learningRate = $(stepSize)
+
+val wordsPerPartition = totalWordCount / $(numPartitions)
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+for {iteration <- 1 to $(maxIter)} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex{ case (i_, 
iter) =>
+logInfo(s"Iteration: $iteration, Partition: $i_")
+logInfo(s"Numerical lib class being used : 
${blas.getClass.getName}")
+val random = new XORShiftRandom(seed ^ ((i_ + 1) << 16) ^ 
((-iteration - 1) << 8))
+val contextWordPairs = iter.flatMap(generateContextWordPairs(_, 
window, random))
+
+val groupedBatches = contextWordPairs.grouped(batchSize)
+
+val negLabels = 1.0f +: 

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114529331
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
+}.repartition($(numPartitions)).cache()
+
+val learningRate = $(stepSize)
+
+val wordsPerPartition = totalWordCount / $(numPartitions)
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+for {iteration <- 1 to $(maxIter)} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex{ case (i_, 
iter) =>
+logInfo(s"Iteration: $iteration, Partition: $i_")
+logInfo(s"Numerical lib class being used : 
${blas.getClass.getName}")
--- End diff --

Yeah, I think `logDebug`


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

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114529246
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
--- End diff --

There should be some level of overlap where we can re-use as much code as 
possible.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114529142
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
--- End diff --

If we wrap up the algo in a separate object, and feed in the `Param` values 
as args, we should only capture the object scope and not need to bc all the 
little params?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114528869
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
--- End diff --

nit: space around `{`: `flatMap { sentence => `


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114528638
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
--- End diff --

If it's easier to add it to mllib Word2Vec that's ok. Preference is not to 
touch mllib if possible - because we would actually prefer to port the older 
mllib stuff over to live purely in ml (and to use DataFrame operations where it 
can potentially improve performance).

But I agree it should probably live in a separate object at the least.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114527850
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
--- End diff --

The final thing returned is the actual vocab sorted by occurrence, so 
`vocab` is probably better?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-05-03 Thread MLnick
Github user MLnick commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r114526827
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -102,6 +107,24 @@ private[feature] trait Word2VecBase extends Params
 "be divided into chunks up to the size (> 0)", ParamValidators.gt(0))
   setDefault(maxSentenceLength -> 1000)
 
+  /**
+   * Choose the technique to use for generating word embeddings
+   * Default: true (Use Skip-Gram with Hierarchical softmax)
+   * @group param
+   */
+  final val skipGram = new BooleanParam(this, "skipGram", "Use Skip-Gram 
model to generate word " +
--- End diff --

We could use `HasSolver` trait here 


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-04-26 Thread Krimit
Github user Krimit commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r113605788
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
--- End diff --

I might be missing something, but what is meant by ``digitSentences``?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-04-26 Thread Krimit
Github user Krimit commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r113605682
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
+}.repartition($(numPartitions)).cache()
+
+val learningRate = $(stepSize)
+
+val wordsPerPartition = totalWordCount / $(numPartitions)
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+for {iteration <- 1 to $(maxIter)} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex{ case (i_, 
iter) =>
+logInfo(s"Iteration: $iteration, Partition: $i_")
+logInfo(s"Numerical lib class being used : 
${blas.getClass.getName}")
+val random = new XORShiftRandom(seed ^ ((i_ + 1) << 16) ^ 
((-iteration - 1) << 8))
+val contextWordPairs = iter.flatMap(generateContextWordPairs(_, 
window, random))
+
+val groupedBatches = contextWordPairs.grouped(batchSize)
+
+val negLabels = 1.0f +: 

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-04-26 Thread Krimit
Github user Krimit commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r113605031
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
+}.repartition($(numPartitions)).cache()
+
+val learningRate = $(stepSize)
+
+val wordsPerPartition = totalWordCount / $(numPartitions)
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+for {iteration <- 1 to $(maxIter)} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex{ case (i_, 
iter) =>
+logInfo(s"Iteration: $iteration, Partition: $i_")
+logInfo(s"Numerical lib class being used : 
${blas.getClass.getName}")
+val random = new XORShiftRandom(seed ^ ((i_ + 1) << 16) ^ 
((-iteration - 1) << 8))
+val contextWordPairs = iter.flatMap(generateContextWordPairs(_, 
window, random))
+
+val groupedBatches = contextWordPairs.grouped(batchSize)
+
+val negLabels = 1.0f +: 

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-04-26 Thread Krimit
Github user Krimit commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r113605000
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
+}.repartition($(numPartitions)).cache()
+
+val learningRate = $(stepSize)
+
+val wordsPerPartition = totalWordCount / $(numPartitions)
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+for {iteration <- 1 to $(maxIter)} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex{ case (i_, 
iter) =>
+logInfo(s"Iteration: $iteration, Partition: $i_")
+logInfo(s"Numerical lib class being used : 
${blas.getClass.getName}")
+val random = new XORShiftRandom(seed ^ ((i_ + 1) << 16) ^ 
((-iteration - 1) << 8))
+val contextWordPairs = iter.flatMap(generateContextWordPairs(_, 
window, random))
+
+val groupedBatches = contextWordPairs.grouped(batchSize)
+
+val negLabels = 1.0f +: 

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-04-26 Thread Krimit
Github user Krimit commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r113603661
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
+}.repartition($(numPartitions)).cache()
+
+val learningRate = $(stepSize)
+
+val wordsPerPartition = totalWordCount / $(numPartitions)
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+for {iteration <- 1 to $(maxIter)} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex{ case (i_, 
iter) =>
+logInfo(s"Iteration: $iteration, Partition: $i_")
+logInfo(s"Numerical lib class being used : 
${blas.getClass.getName}")
+val random = new XORShiftRandom(seed ^ ((i_ + 1) << 16) ^ 
((-iteration - 1) << 8))
+val contextWordPairs = iter.flatMap(generateContextWordPairs(_, 
window, random))
+
+val groupedBatches = contextWordPairs.grouped(batchSize)
+
+val negLabels = 1.0f +: 

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-04-26 Thread Krimit
Github user Krimit commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r113603493
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
+}.repartition($(numPartitions)).cache()
+
+val learningRate = $(stepSize)
+
+val wordsPerPartition = totalWordCount / $(numPartitions)
+
+logInfo(s"VocabSize: ${vocabMap.size}, TotalWordCount: 
$totalWordCount")
+
+for {iteration <- 1 to $(maxIter)} {
+  logInfo(s"Starting iteration: $iteration")
+  val iterationStartTime = System.nanoTime()
+
+  val syn0bc = sc.broadcast(syn0Global)
+  val syn1bc = sc.broadcast(syn1Global)
+
+  val partialFits = digitSentences.mapPartitionsWithIndex{ case (i_, 
iter) =>
+logInfo(s"Iteration: $iteration, Partition: $i_")
+logInfo(s"Numerical lib class being used : 
${blas.getClass.getName}")
--- End diff --

Are these logs really necessary?


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

[GitHub] spark pull request #17673: [SPARK-20372] [ML] Word2Vec Continuous Bag of Wor...

2017-04-26 Thread Krimit
Github user Krimit commented on a diff in the pull request:

https://github.com/apache/spark/pull/17673#discussion_r113603142
  
--- Diff: mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala 
---
@@ -194,6 +232,285 @@ final class Word2Vec @Since("1.4.0") (
 
   @Since("1.4.1")
   override def copy(extra: ParamMap): Word2Vec = defaultCopy(extra)
+
+  /**
+   * Similar to InitUnigramTable in the original code. Instead of using an 
array of size 100 million
+   * like the original, we size it to be 20 times the vocabulary size.
+   * We sacrifice memory here, to get constant time lookups into this 
array when generating
+   * negative samples.
+   */
+  private def generateUnigramTable(normalizedWeights: Array[Double], 
tableSize: Int): Array[Int] = {
+val table = Array.fill(tableSize)(0)
+var a = 0
+var i = 0
+while (a < table.length) {
+  table.update(a, i)
+  if (a.toFloat / table.length >= normalizedWeights(i)) {
+i = math.min(normalizedWeights.length - 1, i + 1)
+  }
+  a += 1
+}
+table
+  }
+
+  private def generateVocab[S <: Iterable[String]](input: RDD[S]):
+  (Int, Long, Map[String, Int], Array[Int]) = {
+val sc = input.context
+
+val words = input.flatMap(x => x)
+
+val vocab = words.map(w => (w, 1L))
+  .reduceByKey(_ + _)
+  .filter{case (w, c) => c >= $(minCount)}
+  .collect()
+  .sortWith{case ((w1, c1), (w2, c2)) => c1 > c2}
+
+val totalWordCount = vocab.map(_._2).sum
+
+val vocabMap = vocab.zipWithIndex.map{case ((w, c), i) =>
+  w -> i
+}.toMap
+
+// We create a cumulative distribution array, unlike the original 
implemention
+// and use binary search to get insertion points. This should 
replicate the same
+// behavior as the table in original implementation.
+val weights = vocab.map(x => scala.math.pow(x._2, power))
+val totalWeight = weights.sum
+
+val normalizedCumWeights = weights.scanLeft(0.0)(_ + _).tail.map(x => 
(x / totalWeight))
+
+val unigramTableSize =
+  math.min(maxUnigramTableSize, unigramTableSizeFactor * 
normalizedCumWeights.length)
+val unigramTable = generateUnigramTable(normalizedCumWeights, 
unigramTableSize)
+
+(vocabMap.size, totalWordCount, vocabMap, unigramTable)
+  }
+
+  /**
+   * This method implements Word2Vec Continuous Bag Of Words based 
implementation using
+   * negative sampling optimization, using BLAS for vectorizing operations 
where applicable.
+   * The algorithm is parallelized in the same way as the skip-gram based 
estimation.
+   * @param input
+   * @return
+   */
+  private def fitCBOW[S <: Iterable[String]](input: RDD[S]): 
feature.Word2VecModel = {
+val (vocabSize, totalWordCount, vocabMap, uniTable) = 
generateVocab(input)
+val negSamples = $(negativeSamples)
+assert(negSamples < vocabSize,
+  s"Vocab size ($vocabSize) cannot be smaller than negative 
samples($negSamples)")
+val seed = $(this.seed)
+val initRandom = new XORShiftRandom(seed)
+
+val vectorSize = $(this.vectorSize)
+
+val syn0Global = Array.fill(vocabSize * 
vectorSize)(initRandom.nextFloat - 0.5f)
+val syn1Global = Array.fill(vocabSize * vectorSize)(0.0f)
+
+val sc = input.context
+
+val vocabMapbc = sc.broadcast(vocabMap)
+val unigramTablebc = sc.broadcast(uniTable)
+
+val window = $(windowSize)
+
+val digitSentences = input.flatMap{sentence =>
+  val wordIndexes = sentence.flatMap(vocabMapbc.value.get)
+  wordIndexes.grouped($(maxSentenceLength)).map(_.toArray)
--- End diff --

I think you need to broadcast ``maxSentenceLength``


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



  1   2   >