aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/mllib/util.py
blob: ed6fd4bca4c548634713678bb0d1163bb6d3ed9c (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
#
# 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.
#

import sys
import numpy as np
import warnings

if sys.version > '3':
    xrange = range
    basestring = str

from pyspark import SparkContext, since
from pyspark.mllib.common import callMLlibFunc, inherit_doc
from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector
from pyspark.sql import DataFrame


class MLUtils(object):

    """
    Helper methods to load, save and pre-process data used in MLlib.

    .. versionadded:: 1.0.0
    """

    @staticmethod
    def _parse_libsvm_line(line, multiclass=None):
        """
        Parses a line in LIBSVM format into (label, indices, values).
        """
        if multiclass is not None:
            warnings.warn("deprecated", DeprecationWarning)
        items = line.split(None)
        label = float(items[0])
        nnz = len(items) - 1
        indices = np.zeros(nnz, dtype=np.int32)
        values = np.zeros(nnz)
        for i in xrange(nnz):
            index, value = items[1 + i].split(":")
            indices[i] = int(index) - 1
            values[i] = float(value)
        return label, indices, values

    @staticmethod
    def _convert_labeled_point_to_libsvm(p):
        """Converts a LabeledPoint to a string in LIBSVM format."""
        from pyspark.mllib.regression import LabeledPoint
        assert isinstance(p, LabeledPoint)
        items = [str(p.label)]
        v = _convert_to_vector(p.features)
        if isinstance(v, SparseVector):
            nnz = len(v.indices)
            for i in xrange(nnz):
                items.append(str(v.indices[i] + 1) + ":" + str(v.values[i]))
        else:
            for i in xrange(len(v)):
                items.append(str(i + 1) + ":" + str(v[i]))
        return " ".join(items)

    @staticmethod
    @since("1.0.0")
    def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None):
        """
        Loads labeled data in the LIBSVM format into an RDD of
        LabeledPoint. The LIBSVM format is a text-based format used by
        LIBSVM and LIBLINEAR. Each line represents a labeled sparse
        feature vector using the following format:

        label index1:value1 index2:value2 ...

        where the indices are one-based and in ascending order. This
        method parses each line into a LabeledPoint, where the feature
        indices are converted to zero-based.

        :param sc: Spark context
        :param path: file or directory path in any Hadoop-supported file
                     system URI
        :param numFeatures: number of features, which will be determined
                            from the input data if a nonpositive value
                            is given. This is useful when the dataset is
                            already split into multiple files and you
                            want to load them separately, because some
                            features may not present in certain files,
                            which leads to inconsistent feature
                            dimensions.
        :param minPartitions: min number of partitions
        @return: labeled data stored as an RDD of LabeledPoint

        >>> from tempfile import NamedTemporaryFile
        >>> from pyspark.mllib.util import MLUtils
        >>> from pyspark.mllib.regression import LabeledPoint
        >>> tempFile = NamedTemporaryFile(delete=True)
        >>> _ = tempFile.write(b"+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0")
        >>> tempFile.flush()
        >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect()
        >>> tempFile.close()
        >>> examples[0]
        LabeledPoint(1.0, (6,[0,2,4],[1.0,2.0,3.0]))
        >>> examples[1]
        LabeledPoint(-1.0, (6,[],[]))
        >>> examples[2]
        LabeledPoint(-1.0, (6,[1,3,5],[4.0,5.0,6.0]))
        """
        from pyspark.mllib.regression import LabeledPoint
        if multiclass is not None:
            warnings.warn("deprecated", DeprecationWarning)

        lines = sc.textFile(path, minPartitions)
        parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l))
        if numFeatures <= 0:
            parsed.cache()
            numFeatures = parsed.map(lambda x: -1 if x[1].size == 0 else x[1][-1]).reduce(max) + 1
        return parsed.map(lambda x: LabeledPoint(x[0], Vectors.sparse(numFeatures, x[1], x[2])))

    @staticmethod
    @since("1.0.0")
    def saveAsLibSVMFile(data, dir):
        """
        Save labeled data in LIBSVM format.

        :param data: an RDD of LabeledPoint to be saved
        :param dir: directory to save the data

        >>> from tempfile import NamedTemporaryFile
        >>> from fileinput import input
        >>> from pyspark.mllib.regression import LabeledPoint
        >>> from glob import glob
        >>> from pyspark.mllib.util import MLUtils
        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])),
        ...             LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
        >>> tempFile = NamedTemporaryFile(delete=True)
        >>> tempFile.close()
        >>> MLUtils.saveAsLibSVMFile(sc.parallelize(examples), tempFile.name)
        >>> ''.join(sorted(input(glob(tempFile.name + "/part-0000*"))))
        '0.0 1:1.01 2:2.02 3:3.03\\n1.1 1:1.23 3:4.56\\n'
        """
        lines = data.map(lambda p: MLUtils._convert_labeled_point_to_libsvm(p))
        lines.saveAsTextFile(dir)

    @staticmethod
    @since("1.1.0")
    def loadLabeledPoints(sc, path, minPartitions=None):
        """
        Load labeled points saved using RDD.saveAsTextFile.

        :param sc: Spark context
        :param path: file or directory path in any Hadoop-supported file
                     system URI
        :param minPartitions: min number of partitions
        @return: labeled data stored as an RDD of LabeledPoint

        >>> from tempfile import NamedTemporaryFile
        >>> from pyspark.mllib.util import MLUtils
        >>> from pyspark.mllib.regression import LabeledPoint
        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])),
        ...             LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
        >>> tempFile = NamedTemporaryFile(delete=True)
        >>> tempFile.close()
        >>> sc.parallelize(examples, 1).saveAsTextFile(tempFile.name)
        >>> MLUtils.loadLabeledPoints(sc, tempFile.name).collect()
        [LabeledPoint(1.1, (3,[0,2],[-1.23,4.56e-07])), LabeledPoint(0.0, [1.01,2.02,3.03])]
        """
        minPartitions = minPartitions or min(sc.defaultParallelism, 2)
        return callMLlibFunc("loadLabeledPoints", sc, path, minPartitions)

    @staticmethod
    @since("1.5.0")
    def appendBias(data):
        """
        Returns a new vector with `1.0` (bias) appended to
        the end of the input vector.
        """
        vec = _convert_to_vector(data)
        if isinstance(vec, SparseVector):
            newIndices = np.append(vec.indices, len(vec))
            newValues = np.append(vec.values, 1.0)
            return SparseVector(len(vec) + 1, newIndices, newValues)
        else:
            return _convert_to_vector(np.append(vec.toArray(), 1.0))

    @staticmethod
    @since("1.5.0")
    def loadVectors(sc, path):
        """
        Loads vectors saved using `RDD[Vector].saveAsTextFile`
        with the default number of partitions.
        """
        return callMLlibFunc("loadVectors", sc, path)

    @staticmethod
    @since("2.0.0")
    def convertVectorColumnsToML(dataset, *cols):
        """
        Converts vector columns in an input DataFrame from the
        :py:class:`pyspark.mllib.linalg.Vector` type to the new
        :py:class:`pyspark.ml.linalg.Vector` type under the `spark.ml`
        package.

        :param dataset:
          input dataset
        :param cols:
          a list of vector columns to be converted.
          New vector columns will be ignored. If unspecified, all old
          vector columns will be converted excepted nested ones.
        :return:
          the input dataset with old vector columns converted to the
          new vector type

        >>> import pyspark
        >>> from pyspark.mllib.linalg import Vectors
        >>> from pyspark.mllib.util import MLUtils
        >>> df = spark.createDataFrame(
        ...     [(0, Vectors.sparse(2, [1], [1.0]), Vectors.dense(2.0, 3.0))],
        ...     ["id", "x", "y"])
        >>> r1 = MLUtils.convertVectorColumnsToML(df).first()
        >>> isinstance(r1.x, pyspark.ml.linalg.SparseVector)
        True
        >>> isinstance(r1.y, pyspark.ml.linalg.DenseVector)
        True
        >>> r2 = MLUtils.convertVectorColumnsToML(df, "x").first()
        >>> isinstance(r2.x, pyspark.ml.linalg.SparseVector)
        True
        >>> isinstance(r2.y, pyspark.mllib.linalg.DenseVector)
        True
        """
        if not isinstance(dataset, DataFrame):
            raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset)))
        return callMLlibFunc("convertVectorColumnsToML", dataset, list(cols))

    @staticmethod
    @since("2.0.0")
    def convertVectorColumnsFromML(dataset, *cols):
        """
        Converts vector columns in an input DataFrame to the
        :py:class:`pyspark.mllib.linalg.Vector` type from the new
        :py:class:`pyspark.ml.linalg.Vector` type under the `spark.ml`
        package.

        :param dataset:
          input dataset
        :param cols:
          a list of vector columns to be converted.
          Old vector columns will be ignored. If unspecified, all new
          vector columns will be converted except nested ones.
        :return:
          the input dataset with new vector columns converted to the
          old vector type

        >>> import pyspark
        >>> from pyspark.ml.linalg import Vectors
        >>> from pyspark.mllib.util import MLUtils
        >>> df = spark.createDataFrame(
        ...     [(0, Vectors.sparse(2, [1], [1.0]), Vectors.dense(2.0, 3.0))],
        ...     ["id", "x", "y"])
        >>> r1 = MLUtils.convertVectorColumnsFromML(df).first()
        >>> isinstance(r1.x, pyspark.mllib.linalg.SparseVector)
        True
        >>> isinstance(r1.y, pyspark.mllib.linalg.DenseVector)
        True
        >>> r2 = MLUtils.convertVectorColumnsFromML(df, "x").first()
        >>> isinstance(r2.x, pyspark.mllib.linalg.SparseVector)
        True
        >>> isinstance(r2.y, pyspark.ml.linalg.DenseVector)
        True
        """
        if not isinstance(dataset, DataFrame):
            raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset)))
        return callMLlibFunc("convertVectorColumnsFromML", dataset, list(cols))

    @staticmethod
    @since("2.0.0")
    def convertMatrixColumnsToML(dataset, *cols):
        """
        Converts matrix columns in an input DataFrame from the
        :py:class:`pyspark.mllib.linalg.Matrix` type to the new
        :py:class:`pyspark.ml.linalg.Matrix` type under the `spark.ml`
        package.

        :param dataset:
          input dataset
        :param cols:
          a list of matrix columns to be converted.
          New matrix columns will be ignored. If unspecified, all old
          matrix columns will be converted excepted nested ones.
        :return:
          the input dataset with old matrix columns converted to the
          new matrix type

        >>> import pyspark
        >>> from pyspark.mllib.linalg import Matrices
        >>> from pyspark.mllib.util import MLUtils
        >>> df = spark.createDataFrame(
        ...     [(0, Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]),
        ...     Matrices.dense(2, 2, range(4)))], ["id", "x", "y"])
        >>> r1 = MLUtils.convertMatrixColumnsToML(df).first()
        >>> isinstance(r1.x, pyspark.ml.linalg.SparseMatrix)
        True
        >>> isinstance(r1.y, pyspark.ml.linalg.DenseMatrix)
        True
        >>> r2 = MLUtils.convertMatrixColumnsToML(df, "x").first()
        >>> isinstance(r2.x, pyspark.ml.linalg.SparseMatrix)
        True
        >>> isinstance(r2.y, pyspark.mllib.linalg.DenseMatrix)
        True
        """
        if not isinstance(dataset, DataFrame):
            raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset)))
        return callMLlibFunc("convertMatrixColumnsToML", dataset, list(cols))

    @staticmethod
    @since("2.0.0")
    def convertMatrixColumnsFromML(dataset, *cols):
        """
        Converts matrix columns in an input DataFrame to the
        :py:class:`pyspark.mllib.linalg.Matrix` type from the new
        :py:class:`pyspark.ml.linalg.Matrix` type under the `spark.ml`
        package.

        :param dataset:
          input dataset
        :param cols:
          a list of matrix columns to be converted.
          Old matrix columns will be ignored. If unspecified, all new
          matrix columns will be converted except nested ones.
        :return:
          the input dataset with new matrix columns converted to the
          old matrix type

        >>> import pyspark
        >>> from pyspark.ml.linalg import Matrices
        >>> from pyspark.mllib.util import MLUtils
        >>> df = spark.createDataFrame(
        ...     [(0, Matrices.sparse(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4]),
        ...     Matrices.dense(2, 2, range(4)))], ["id", "x", "y"])
        >>> r1 = MLUtils.convertMatrixColumnsFromML(df).first()
        >>> isinstance(r1.x, pyspark.mllib.linalg.SparseMatrix)
        True
        >>> isinstance(r1.y, pyspark.mllib.linalg.DenseMatrix)
        True
        >>> r2 = MLUtils.convertMatrixColumnsFromML(df, "x").first()
        >>> isinstance(r2.x, pyspark.mllib.linalg.SparseMatrix)
        True
        >>> isinstance(r2.y, pyspark.ml.linalg.DenseMatrix)
        True
        """
        if not isinstance(dataset, DataFrame):
            raise TypeError("Input dataset must be a DataFrame but got {}.".format(type(dataset)))
        return callMLlibFunc("convertMatrixColumnsFromML", dataset, list(cols))


class Saveable(object):
    """
    Mixin for models and transformers which may be saved as files.

    .. versionadded:: 1.3.0
    """

    def save(self, sc, path):
        """
        Save this model to the given path.

        This saves:
         * human-readable (JSON) model metadata to path/metadata/
         * Parquet formatted data to path/data/

        The model may be loaded using py:meth:`Loader.load`.

        :param sc: Spark context used to save model data.
        :param path: Path specifying the directory in which to save
                     this model. If the directory already exists,
                     this method throws an exception.
        """
        raise NotImplementedError


@inherit_doc
class JavaSaveable(Saveable):
    """
    Mixin for models that provide save() through their Scala
    implementation.

    .. versionadded:: 1.3.0
    """

    @since("1.3.0")
    def save(self, sc, path):
        """Save this model to the given path."""
        if not isinstance(sc, SparkContext):
            raise TypeError("sc should be a SparkContext, got type %s" % type(sc))
        if not isinstance(path, basestring):
            raise TypeError("path should be a basestring, got type %s" % type(path))
        self._java_model.save(sc._jsc.sc(), path)


class Loader(object):
    """
    Mixin for classes which can load saved models from files.

    .. versionadded:: 1.3.0
    """

    @classmethod
    def load(cls, sc, path):
        """
        Load a model from the given path. The model should have been
        saved using py:meth:`Saveable.save`.

        :param sc: Spark context used for loading model files.
        :param path: Path specifying the directory to which the model
                     was saved.
        :return: model instance
        """
        raise NotImplemented


@inherit_doc
class JavaLoader(Loader):
    """
    Mixin for classes which can load saved models using its Scala
    implementation.

    .. versionadded:: 1.3.0
    """

    @classmethod
    def _java_loader_class(cls):
        """
        Returns the full class name of the Java loader. The default
        implementation replaces "pyspark" by "org.apache.spark" in
        the Python full class name.
        """
        java_package = cls.__module__.replace("pyspark", "org.apache.spark")
        return ".".join([java_package, cls.__name__])

    @classmethod
    def _load_java(cls, sc, path):
        """
        Load a Java model from the given path.
        """
        java_class = cls._java_loader_class()
        java_obj = sc._jvm
        for name in java_class.split("."):
            java_obj = getattr(java_obj, name)
        return java_obj.load(sc._jsc.sc(), path)

    @classmethod
    @since("1.3.0")
    def load(cls, sc, path):
        """Load a model from the given path."""
        java_model = cls._load_java(sc, path)
        return cls(java_model)


class LinearDataGenerator(object):
    """Utils for generating linear data.

    .. versionadded:: 1.5.0
    """

    @staticmethod
    @since("1.5.0")
    def generateLinearInput(intercept, weights, xMean, xVariance,
                            nPoints, seed, eps):
        """
        :param: intercept bias factor, the term c in X'w + c
        :param: weights   feature vector, the term w in X'w + c
        :param: xMean     Point around which the data X is centered.
        :param: xVariance Variance of the given data
        :param: nPoints   Number of points to be generated
        :param: seed      Random Seed
        :param: eps       Used to scale the noise. If eps is set high,
                          the amount of gaussian noise added is more.

        Returns a list of LabeledPoints of length nPoints
        """
        weights = [float(weight) for weight in weights]
        xMean = [float(mean) for mean in xMean]
        xVariance = [float(var) for var in xVariance]
        return list(callMLlibFunc(
            "generateLinearInputWrapper", float(intercept), weights, xMean,
            xVariance, int(nPoints), int(seed), float(eps)))

    @staticmethod
    @since("1.5.0")
    def generateLinearRDD(sc, nexamples, nfeatures, eps,
                          nParts=2, intercept=0.0):
        """
        Generate a RDD of LabeledPoints.
        """
        return callMLlibFunc(
            "generateLinearRDDWrapper", sc, int(nexamples), int(nfeatures),
            float(eps), int(nParts), float(intercept))


def _test():
    import doctest
    from pyspark.sql import SparkSession
    globs = globals().copy()
    # The small batch size here ensures that we see multiple batches,
    # even in these small test examples:
    spark = SparkSession.builder\
        .master("local[2]")\
        .appName("mllib.util tests")\
        .getOrCreate()
    globs['spark'] = spark
    globs['sc'] = spark.sparkContext
    (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS)
    spark.stop()
    if failure_count:
        exit(-1)


if __name__ == "__main__":
    _test()