## 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.#fromtypingimportGeneric,List,Optional,Tuple,TypeVar,Unionimportsysfrompysparkimportsincefrompyspark.rddimportRDDfrompyspark.mllib.commonimportJavaModelWrapper,callMLlibFuncfrompyspark.mllib.linalgimportMatrixfrompyspark.sqlimportSQLContextfrompyspark.sql.typesimportArrayType,DoubleType,StructField,StructType__all__=["BinaryClassificationMetrics","RegressionMetrics","MulticlassMetrics","RankingMetrics",]T=TypeVar("T")
[docs]classBinaryClassificationMetrics(JavaModelWrapper):""" Evaluator for binary classification. .. versionadded:: 1.4.0 Parameters ---------- scoreAndLabels : :py:class:`pyspark.RDD` an RDD of score, label and optional weight. Examples -------- >>> scoreAndLabels = sc.parallelize([ ... (0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)], 2) >>> metrics = BinaryClassificationMetrics(scoreAndLabels) >>> metrics.areaUnderROC 0.70... >>> metrics.areaUnderPR 0.83... >>> metrics.unpersist() >>> scoreAndLabelsWithOptWeight = sc.parallelize([ ... (0.1, 0.0, 1.0), (0.1, 1.0, 0.4), (0.4, 0.0, 0.2), (0.6, 0.0, 0.6), (0.6, 1.0, 0.9), ... (0.6, 1.0, 0.5), (0.8, 1.0, 0.7)], 2) >>> metrics = BinaryClassificationMetrics(scoreAndLabelsWithOptWeight) >>> metrics.areaUnderROC 0.79... >>> metrics.areaUnderPR 0.88... """def__init__(self,scoreAndLabels:RDD[Tuple[float,float]]):sc=scoreAndLabels.ctxsql_ctx=SQLContext.getOrCreate(sc)numCol=len(scoreAndLabels.first())schema=StructType([StructField("score",DoubleType(),nullable=False),StructField("label",DoubleType(),nullable=False),])ifnumCol==3:schema.add("weight",DoubleType(),False)df=sql_ctx.createDataFrame(scoreAndLabels,schema=schema)assertsc._jvmisnotNonejava_class=sc._jvm.org.apache.spark.mllib.evaluation.BinaryClassificationMetricsjava_model=java_class(df._jdf)super(BinaryClassificationMetrics,self).__init__(java_model)@property@since("1.4.0")defareaUnderROC(self)->float:""" Computes the area under the receiver operating characteristic (ROC) curve. """returnself.call("areaUnderROC")@property@since("1.4.0")defareaUnderPR(self)->float:""" Computes the area under the precision-recall curve. """returnself.call("areaUnderPR")
[docs]@since("1.4.0")defunpersist(self)->None:""" Unpersists intermediate RDDs used in the computation. """self.call("unpersist")
[docs]classRegressionMetrics(JavaModelWrapper):""" Evaluator for regression. .. versionadded:: 1.4.0 Parameters ---------- predictionAndObservations : :py:class:`pyspark.RDD` an RDD of prediction, observation and optional weight. Examples -------- >>> predictionAndObservations = sc.parallelize([ ... (2.5, 3.0), (0.0, -0.5), (2.0, 2.0), (8.0, 7.0)]) >>> metrics = RegressionMetrics(predictionAndObservations) >>> metrics.explainedVariance 8.859... >>> metrics.meanAbsoluteError 0.5... >>> metrics.meanSquaredError 0.37... >>> metrics.rootMeanSquaredError 0.61... >>> metrics.r2 0.94... >>> predictionAndObservationsWithOptWeight = sc.parallelize([ ... (2.5, 3.0, 0.5), (0.0, -0.5, 1.0), (2.0, 2.0, 0.3), (8.0, 7.0, 0.9)]) >>> metrics = RegressionMetrics(predictionAndObservationsWithOptWeight) >>> metrics.rootMeanSquaredError 0.68... """def__init__(self,predictionAndObservations:RDD[Tuple[float,float]]):sc=predictionAndObservations.ctxsql_ctx=SQLContext.getOrCreate(sc)numCol=len(predictionAndObservations.first())schema=StructType([StructField("prediction",DoubleType(),nullable=False),StructField("observation",DoubleType(),nullable=False),])ifnumCol==3:schema.add("weight",DoubleType(),False)df=sql_ctx.createDataFrame(predictionAndObservations,schema=schema)assertsc._jvmisnotNonejava_class=sc._jvm.org.apache.spark.mllib.evaluation.RegressionMetricsjava_model=java_class(df._jdf)super(RegressionMetrics,self).__init__(java_model)@property@since("1.4.0")defexplainedVariance(self)->float:r""" Returns the explained variance regression score. explainedVariance = :math:`1 - \frac{variance(y - \hat{y})}{variance(y)}` """returnself.call("explainedVariance")@property@since("1.4.0")defmeanAbsoluteError(self)->float:""" Returns the mean absolute error, which is a risk function corresponding to the expected value of the absolute error loss or l1-norm loss. """returnself.call("meanAbsoluteError")@property@since("1.4.0")defmeanSquaredError(self)->float:""" Returns the mean squared error, which is a risk function corresponding to the expected value of the squared error loss or quadratic loss. """returnself.call("meanSquaredError")@property@since("1.4.0")defrootMeanSquaredError(self)->float:""" Returns the root mean squared error, which is defined as the square root of the mean squared error. """returnself.call("rootMeanSquaredError")@property@since("1.4.0")defr2(self)->float:""" Returns R^2^, the coefficient of determination. """returnself.call("r2")
[docs]@since("1.4.0")defconfusionMatrix(self)->Matrix:""" Returns confusion matrix: predicted classes are in columns, they are ordered by class label ascending, as in "labels". """returnself.call("confusionMatrix")
[docs]@since("1.4.0")deftruePositiveRate(self,label:float)->float:""" Returns true positive rate for a given label (category). """returnself.call("truePositiveRate",label)
[docs]@since("1.4.0")deffalsePositiveRate(self,label:float)->float:""" Returns false positive rate for a given label (category). """returnself.call("falsePositiveRate",label)
@property@since("2.0.0")defaccuracy(self)->float:""" Returns accuracy (equals to the total number of correctly classified instances out of the total number of instances). """returnself.call("accuracy")@property@since("1.4.0")defweightedTruePositiveRate(self)->float:""" Returns weighted true positive rate. (equals to precision, recall and f-measure) """returnself.call("weightedTruePositiveRate")@property@since("1.4.0")defweightedFalsePositiveRate(self)->float:""" Returns weighted false positive rate. """returnself.call("weightedFalsePositiveRate")@property@since("1.4.0")defweightedRecall(self)->float:""" Returns weighted averaged recall. (equals to precision, recall and f-measure) """returnself.call("weightedRecall")@property@since("1.4.0")defweightedPrecision(self)->float:""" Returns weighted averaged precision. """returnself.call("weightedPrecision")
[docs]@since("1.4.0")defprecisionAt(self,k:int)->float:""" Compute the average precision of all the queries, truncated at ranking position k. If for a query, the ranking algorithm returns n (n < k) results, the precision value will be computed as #(relevant items retrieved) / k. This formula also applies when the size of the ground truth set is less than k. If a query has an empty ground truth set, zero will be used as precision together with a log warning. """returnself.call("precisionAt",int(k))
@property@since("1.4.0")defmeanAveragePrecision(self)->float:""" Returns the mean average precision (MAP) of all the queries. If a query has an empty ground truth set, the average precision will be zero and a log warning is generated. """returnself.call("meanAveragePrecision")
[docs]@since("3.0.0")defmeanAveragePrecisionAt(self,k:int)->float:""" Returns the mean average precision (MAP) at first k ranking of all the queries. If a query has an empty ground truth set, the average precision will be zero and a log warning is generated. """returnself.call("meanAveragePrecisionAt",int(k))
[docs]@since("1.4.0")defndcgAt(self,k:int)->float:""" Compute the average NDCG value of all the queries, truncated at ranking position k. The discounted cumulative gain at position k is computed as: sum,,i=1,,^k^ (2^{relevance of ''i''th item}^ - 1) / log(i + 1), and the NDCG is obtained by dividing the DCG value on the ground truth set. In the current implementation, the relevance value is binary. If a query has an empty ground truth set, zero will be used as NDCG together with a log warning. """returnself.call("ndcgAt",int(k))
[docs]@since("3.0.0")defrecallAt(self,k:int)->float:""" Compute the average recall of all the queries, truncated at ranking position k. If for a query, the ranking algorithm returns n results, the recall value will be computed as #(relevant items retrieved) / #(ground truth set). This formula also applies when the size of the ground truth set is less than k. If a query has an empty ground truth set, zero will be used as recall together with a log warning. """returnself.call("recallAt",int(k))
classMultilabelMetrics(JavaModelWrapper):""" Evaluator for multilabel classification. .. versionadded:: 1.4.0 Parameters ---------- predictionAndLabels : :py:class:`pyspark.RDD` an RDD of (predictions, labels) pairs, both are non-null Arrays, each with unique elements. Examples -------- >>> predictionAndLabels = sc.parallelize([([0.0, 1.0], [0.0, 2.0]), ([0.0, 2.0], [0.0, 1.0]), ... ([], [0.0]), ([2.0], [2.0]), ([2.0, 0.0], [2.0, 0.0]), ... ([0.0, 1.0, 2.0], [0.0, 1.0]), ([1.0], [1.0, 2.0])]) >>> metrics = MultilabelMetrics(predictionAndLabels) >>> metrics.precision(0.0) 1.0 >>> metrics.recall(1.0) 0.66... >>> metrics.f1Measure(2.0) 0.5 >>> metrics.precision() 0.66... >>> metrics.recall() 0.64... >>> metrics.f1Measure() 0.63... >>> metrics.microPrecision 0.72... >>> metrics.microRecall 0.66... >>> metrics.microF1Measure 0.69... >>> metrics.hammingLoss 0.33... >>> metrics.subsetAccuracy 0.28... >>> metrics.accuracy 0.54... """def__init__(self,predictionAndLabels:RDD[Tuple[List[float],List[float]]]):sc=predictionAndLabels.ctxsql_ctx=SQLContext.getOrCreate(sc)df=sql_ctx.createDataFrame(predictionAndLabels,schema=sql_ctx._inferSchema(predictionAndLabels))assertsc._jvmisnotNonejava_class=sc._jvm.org.apache.spark.mllib.evaluation.MultilabelMetricsjava_model=java_class(df._jdf)super(MultilabelMetrics,self).__init__(java_model)@since("1.4.0")defprecision(self,label:Optional[float]=None)->float:""" Returns precision or precision for a given label (category) if specified. """iflabelisNone:returnself.call("precision")else:returnself.call("precision",float(label))@since("1.4.0")defrecall(self,label:Optional[float]=None)->float:""" Returns recall or recall for a given label (category) if specified. """iflabelisNone:returnself.call("recall")else:returnself.call("recall",float(label))@since("1.4.0")deff1Measure(self,label:Optional[float]=None)->float:""" Returns f1Measure or f1Measure for a given label (category) if specified. """iflabelisNone:returnself.call("f1Measure")else:returnself.call("f1Measure",float(label))@property@since("1.4.0")defmicroPrecision(self)->float:""" Returns micro-averaged label-based precision. (equals to micro-averaged document-based precision) """returnself.call("microPrecision")@property@since("1.4.0")defmicroRecall(self)->float:""" Returns micro-averaged label-based recall. (equals to micro-averaged document-based recall) """returnself.call("microRecall")@property@since("1.4.0")defmicroF1Measure(self)->float:""" Returns micro-averaged label-based f1-measure. (equals to micro-averaged document-based f1-measure) """returnself.call("microF1Measure")@property@since("1.4.0")defhammingLoss(self)->float:""" Returns Hamming-loss. """returnself.call("hammingLoss")@property@since("1.4.0")defsubsetAccuracy(self)->float:""" Returns subset accuracy. (for equal sets of labels) """returnself.call("subsetAccuracy")@property@since("1.4.0")defaccuracy(self)->float:""" Returns accuracy. """returnself.call("accuracy")def_test()->None:importdoctestimportnumpyfrompyspark.sqlimportSparkSessionimportpyspark.mllib.evaluationtry:# Numpy 1.14+ changed it's string format.numpy.set_printoptions(legacy="1.13")exceptTypeError:passglobs=pyspark.mllib.evaluation.__dict__.copy()spark=SparkSession.builder.master("local[4]").appName("mllib.evaluation tests").getOrCreate()globs["sc"]=spark.sparkContext(failure_count,test_count)=doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS)spark.stop()iffailure_count:sys.exit(-1)if__name__=="__main__":_test()