Skip to content

Commit e87d973

Browse files
author
Robert Kruszewski
committed
Revert "[SPARK-26616][MLLIB] Expose document frequency in IDFModel"
This reverts commit d2e86cb.
1 parent b28e76a commit e87d973

File tree

8 files changed

+20
-91
lines changed

8 files changed

+20
-91
lines changed

mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala

Lines changed: 6 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@ import org.apache.spark.rdd.RDD
3232
import org.apache.spark.sql._
3333
import org.apache.spark.sql.functions._
3434
import org.apache.spark.sql.types.StructType
35-
import org.apache.spark.util.VersionUtils.majorVersion
3635

3736
/**
3837
* Params for [[IDF]] and [[IDFModel]].
@@ -152,15 +151,6 @@ class IDFModel private[ml] (
152151
@Since("2.0.0")
153152
def idf: Vector = idfModel.idf.asML
154153

155-
/** Returns the document frequency */
156-
@Since("3.0.0")
157-
def docFreq: Array[Long] = idfModel.docFreq
158-
159-
/** Returns number of documents evaluated to compute idf */
160-
@Since("3.0.0")
161-
def numDocs: Long = idfModel.numDocs
162-
163-
164154
@Since("1.6.0")
165155
override def write: MLWriter = new IDFModelWriter(this)
166156
}
@@ -170,11 +160,11 @@ object IDFModel extends MLReadable[IDFModel] {
170160

171161
private[IDFModel] class IDFModelWriter(instance: IDFModel) extends MLWriter {
172162

173-
private case class Data(idf: Vector, docFreq: Array[Long], numDocs: Long)
163+
private case class Data(idf: Vector)
174164

175165
override protected def saveImpl(path: String): Unit = {
176166
DefaultParamsWriter.saveMetadata(instance, path, sc)
177-
val data = Data(instance.idf, instance.docFreq, instance.numDocs)
167+
val data = Data(instance.idf)
178168
val dataPath = new Path(path, "data").toString
179169
sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath)
180170
}
@@ -188,19 +178,10 @@ object IDFModel extends MLReadable[IDFModel] {
188178
val metadata = DefaultParamsReader.loadMetadata(path, sc, className)
189179
val dataPath = new Path(path, "data").toString
190180
val data = sparkSession.read.parquet(dataPath)
191-
192-
val model = if (majorVersion(metadata.sparkVersion) >= 3) {
193-
val Row(idf: Vector, df: Seq[_], numDocs: Long) = data.select("idf", "docFreq", "numDocs")
194-
.head()
195-
new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf),
196-
df.asInstanceOf[Seq[Long]].toArray, numDocs))
197-
} else {
198-
val Row(idf: Vector) = MLUtils.convertVectorColumnsToML(data, "idf")
199-
.select("idf")
200-
.head()
201-
new IDFModel(metadata.uid,
202-
new feature.IDFModel(OldVectors.fromML(idf), new Array[Long](idf.size), 0L))
203-
}
181+
val Row(idf: Vector) = MLUtils.convertVectorColumnsToML(data, "idf")
182+
.select("idf")
183+
.head()
184+
val model = new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf)))
204185
metadata.getAndSetParams(model)
205186
model
206187
}

mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala

Lines changed: 7 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,6 @@ import org.apache.spark.rdd.RDD
3232
* This implementation supports filtering out terms which do not appear in a minimum number
3333
* of documents (controlled by the variable `minDocFreq`). For terms that are not in
3434
* at least `minDocFreq` documents, the IDF is found as 0, resulting in TF-IDFs of 0.
35-
* The document frequency is 0 as well for such terms
3635
*
3736
* @param minDocFreq minimum of documents in which a term
3837
* should appear for filtering
@@ -51,12 +50,12 @@ class IDF @Since("1.2.0") (@Since("1.2.0") val minDocFreq: Int) {
5150
*/
5251
@Since("1.1.0")
5352
def fit(dataset: RDD[Vector]): IDFModel = {
54-
val (idf: Vector, docFreq: Array[Long], numDocs: Long) = dataset.treeAggregate(
55-
new IDF.DocumentFrequencyAggregator(minDocFreq = minDocFreq))(
53+
val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator(
54+
minDocFreq = minDocFreq))(
5655
seqOp = (df, v) => df.add(v),
5756
combOp = (df1, df2) => df1.merge(df2)
5857
).idf()
59-
new IDFModel(idf, docFreq, numDocs)
58+
new IDFModel(idf)
6059
}
6160

6261
/**
@@ -129,14 +128,13 @@ private object IDF {
129128

130129
private def isEmpty: Boolean = m == 0L
131130

132-
/** Returns the current IDF vector, docFreq, number of documents */
133-
def idf(): (Vector, Array[Long], Long) = {
131+
/** Returns the current IDF vector. */
132+
def idf(): Vector = {
134133
if (isEmpty) {
135134
throw new IllegalStateException("Haven't seen any document yet.")
136135
}
137136
val n = df.length
138137
val inv = new Array[Double](n)
139-
val dfv = new Array[Long](n)
140138
var j = 0
141139
while (j < n) {
142140
/*
@@ -150,11 +148,10 @@ private object IDF {
150148
*/
151149
if (df(j) >= minDocFreq) {
152150
inv(j) = math.log((m + 1.0) / (df(j) + 1.0))
153-
dfv(j) = df(j)
154151
}
155152
j += 1
156153
}
157-
(Vectors.dense(inv), dfv, m)
154+
Vectors.dense(inv)
158155
}
159156
}
160157
}
@@ -163,9 +160,7 @@ private object IDF {
163160
* Represents an IDF model that can transform term frequency vectors.
164161
*/
165162
@Since("1.1.0")
166-
class IDFModel private[spark](@Since("1.1.0") val idf: Vector,
167-
@Since("3.0.0") val docFreq: Array[Long],
168-
@Since("3.0.0") val numDocs: Long) extends Serializable {
163+
class IDFModel private[spark] (@Since("1.1.0") val idf: Vector) extends Serializable {
169164

170165
/**
171166
* Transforms term frequency (TF) vectors to TF-IDF vectors.

mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ class IDFSuite extends MLTest with DefaultReadWriteTest {
4444

4545
test("params") {
4646
ParamsSuite.checkParams(new IDF)
47-
val model = new IDFModel("idf", new OldIDFModel(Vectors.dense(1.0), Array(1L), 1))
47+
val model = new IDFModel("idf", new OldIDFModel(Vectors.dense(1.0)))
4848
ParamsSuite.checkParams(model)
4949
}
5050

@@ -112,13 +112,10 @@ class IDFSuite extends MLTest with DefaultReadWriteTest {
112112
}
113113

114114
test("IDFModel read/write") {
115-
val instance = new IDFModel("myIDFModel",
116-
new OldIDFModel(Vectors.dense(1.0, 2.0), Array(1, 2), 2))
115+
val instance = new IDFModel("myIDFModel", new OldIDFModel(Vectors.dense(1.0, 2.0)))
117116
.setInputCol("myInputCol")
118117
.setOutputCol("myOutputCol")
119118
val newInstance = testDefaultReadWrite(instance)
120119
assert(newInstance.idf === instance.idf)
121-
assert(newInstance.docFreq === instance.docFreq)
122-
assert(newInstance.numDocs === instance.numDocs)
123120
}
124121
}

mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala

Lines changed: 4 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -39,11 +39,9 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext {
3939
math.log((m + 1.0) / (x + 1.0))
4040
})
4141
assert(model.idf ~== expected absTol 1e-12)
42-
assert(model.numDocs === 3)
43-
assert(model.docFreq === Array(0, 3, 1, 2))
4442

4543
val assertHelper = (tfidf: Array[Vector]) => {
46-
assert(tfidf.length === 3)
44+
assert(tfidf.size === 3)
4745
val tfidf0 = tfidf(0).asInstanceOf[SparseVector]
4846
assert(tfidf0.indices === Array(1, 3))
4947
assert(Vectors.dense(tfidf0.values) ~==
@@ -72,21 +70,19 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext {
7270
)
7371
val m = localTermFrequencies.size
7472
val termFrequencies = sc.parallelize(localTermFrequencies, 2)
75-
val idf = new IDF(minDocFreq = 2)
73+
val idf = new IDF(minDocFreq = 1)
7674
val model = idf.fit(termFrequencies)
7775
val expected = Vectors.dense(Array(0, 3, 1, 2).map { x =>
78-
if (x >= 2) {
76+
if (x > 0) {
7977
math.log((m + 1.0) / (x + 1.0))
8078
} else {
8179
0
8280
}
8381
})
8482
assert(model.idf ~== expected absTol 1e-12)
85-
assert(model.numDocs === 3)
86-
assert(model.docFreq === Array(0, 3, 0, 2))
8783

8884
val assertHelper = (tfidf: Array[Vector]) => {
89-
assert(tfidf.length === 3)
85+
assert(tfidf.size === 3)
9086
val tfidf0 = tfidf(0).asInstanceOf[SparseVector]
9187
assert(tfidf0.indices === Array(1, 3))
9288
assert(Vectors.dense(tfidf0.values) ~==

project/MimaExcludes.scala

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -165,11 +165,7 @@ object MimaExcludes {
165165
case ReversedMissingMethodProblem(meth) =>
166166
!meth.owner.fullName.startsWith("org.apache.spark.sql.sources.v2")
167167
case _ => true
168-
},
169-
170-
// [SPARK-26616][MLlib] Expose document frequency in IDFModel
171-
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"),
172-
ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf")
168+
}
173169
)
174170

175171
// Exclude rules for 2.4.x

python/pyspark/ml/feature.py

Lines changed: 0 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -967,10 +967,6 @@ class IDF(JavaEstimator, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWritab
967967
>>> model = idf.fit(df)
968968
>>> model.idf
969969
DenseVector([0.0, 0.0])
970-
>>> model.docFreq
971-
[0, 3]
972-
>>> model.numDocs == df.count()
973-
True
974970
>>> model.transform(df).head().idf
975971
DenseVector([0.0, 0.0])
976972
>>> idf.setParams(outputCol="freqs").fit(df).transform(df).collect()[1].freqs
@@ -1050,22 +1046,6 @@ def idf(self):
10501046
"""
10511047
return self._call_java("idf")
10521048

1053-
@property
1054-
@since("3.0.0")
1055-
def docFreq(self):
1056-
"""
1057-
Returns the document frequency.
1058-
"""
1059-
return self._call_java("docFreq")
1060-
1061-
@property
1062-
@since("3.0.0")
1063-
def numDocs(self):
1064-
"""
1065-
Returns number of documents evaluated to compute idf
1066-
"""
1067-
return self._call_java("numDocs")
1068-
10691049

10701050
@inherit_doc
10711051
class Imputer(JavaEstimator, HasInputCols, JavaMLReadable, JavaMLWritable):

python/pyspark/ml/tests/test_feature.py

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -67,8 +67,6 @@ def test_idf(self):
6767
"Model should inherit the UID from its parent estimator.")
6868
output = idf0m.transform(dataset)
6969
self.assertIsNotNone(output.head().idf)
70-
self.assertIsNotNone(idf0m.docFreq)
71-
self.assertEqual(idf0m.numDocs, 3)
7270
# Test that parameters transferred to Python Model
7371
check_params(self, idf0m)
7472

python/pyspark/mllib/feature.py

Lines changed: 0 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -518,20 +518,6 @@ def idf(self):
518518
"""
519519
return self.call('idf')
520520

521-
@since('3.0.0')
522-
def docFreq(self):
523-
"""
524-
Returns the document frequency.
525-
"""
526-
return self.call('docFreq')
527-
528-
@since('3.0.0')
529-
def numDocs(self):
530-
"""
531-
Returns number of documents evaluated to compute idf
532-
"""
533-
return self.call('numDocs')
534-
535521

536522
class IDF(object):
537523
"""

0 commit comments

Comments
 (0)