Skip to content

Commit

Permalink
[SPARK-28855][CORE][ML][SQL][STREAMING] Remove outdated usages of Exp…
Browse files Browse the repository at this point in the history
…erimental, Evolving annotations

### What changes were proposed in this pull request?

The Experimental and Evolving annotations are both (like Unstable) used to express that a an API may change. However there are many things in the code that have been marked that way since even Spark 1.x. Per the dev thread, anything introduced at or before Spark 2.3.0 is pretty much 'stable' in that it would not change without a deprecation cycle. Therefore I'd like to remove most of these annotations. And, remove the `:: Experimental ::` scaladoc tag too. And likewise for Python, R.

The changes below can be summarized as:
- Generally, anything introduced at or before Spark 2.3.0 has been unmarked as neither Evolving nor Experimental
- Obviously experimental items like DSv2, Barrier mode, ExperimentalMethods are untouched
- I _did_ unmark a few MLlib classes introduced in 2.4, as I am quite confident they're not going to change (e.g. KolmogorovSmirnovTest, PowerIterationClustering)

It's a big change to review, so I'd suggest scanning the list of _files_ changed to see if any area seems like it should remain partly experimental and examine those.

### Why are the changes needed?

Many of these annotations are incorrect; the APIs are de facto stable. Leaving them also makes legitimate usages of the annotations less meaningful.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #25558 from srowen/SPARK-28855.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
  • Loading branch information
srowen committed Sep 1, 2019
1 parent 3821d75 commit eb037a8
Show file tree
Hide file tree
Showing 76 changed files with 39 additions and 552 deletions.
Expand Up @@ -19,8 +19,6 @@ package org.apache.spark.metrics.source

import com.codahale.metrics.MetricRegistry

import org.apache.spark.annotation.Experimental

private[spark] object StaticSources {
/**
* The set of all static sources. These sources may be reported to from any class, including
Expand All @@ -30,10 +28,8 @@ private[spark] object StaticSources {
}

/**
* :: Experimental ::
* Metrics for code generation.
*/
@Experimental
object CodegenMetrics extends Source {
override val sourceName: String = "CodeGenerator"
override val metricRegistry: MetricRegistry = new MetricRegistry()
Expand Down Expand Up @@ -62,10 +58,8 @@ object CodegenMetrics extends Source {
}

/**
* :: Experimental ::
* Metrics for access to the hive external catalog.
*/
@Experimental
object HiveCatalogMetrics extends Source {
override val sourceName: String = "HiveExternalCatalog"
override val metricRegistry: MetricRegistry = new MetricRegistry()
Expand Down
2 changes: 0 additions & 2 deletions core/src/main/scala/org/apache/spark/partial/package.scala
Expand Up @@ -18,8 +18,6 @@
package org.apache.spark

/**
* :: Experimental ::
*
* Support for approximate results. This provides convenient api and also implementation for
* approximate calculation.
*
Expand Down
Expand Up @@ -34,7 +34,6 @@ import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewO

import org.apache.spark._
import org.apache.spark.Partitioner.defaultPartitioner
import org.apache.spark.annotation.Experimental
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.SPECULATION_ENABLED
import org.apache.spark.internal.io._
Expand All @@ -52,7 +51,6 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
extends Logging with Serializable {

/**
* :: Experimental ::
* Generic function to combine the elements for each key using a custom set of aggregation
* functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C
*
Expand All @@ -68,7 +66,6 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
* @note V and C can be different -- for example, one might group an RDD of type
* (Int, Int) into an RDD of type (Int, Seq[Int]).
*/
@Experimental
def combineByKeyWithClassTag[C](
createCombiner: V => C,
mergeValue: (C, V) => C,
Expand Down Expand Up @@ -136,10 +133,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
}

/**
* :: Experimental ::
* Simplified version of combineByKeyWithClassTag that hash-partitions the output RDD.
*/
@Experimental
def combineByKeyWithClassTag[C](
createCombiner: V => C,
mergeValue: (C, V) => C,
Expand Down Expand Up @@ -616,11 +611,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
}

/**
* :: Experimental ::
* Simplified version of combineByKeyWithClassTag that hash-partitions the resulting RDD using the
* existing partitioner/parallelism level.
*/
@Experimental
def combineByKeyWithClassTag[C](
createCombiner: V => C,
mergeValue: (C, V) => C,
Expand Down
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark.sql

import org.apache.spark.annotation.Experimental

package object avro {

/**
Expand All @@ -31,7 +29,6 @@ package object avro {
*
* @since 2.4.0
*/
@Experimental
@deprecated("Please use 'org.apache.spark.sql.avro.functions.from_avro' instead.", "3.0.0")
def from_avro(
data: Column,
Expand All @@ -45,7 +42,6 @@ package object avro {
*
* @since 2.4.0
*/
@Experimental
@deprecated("Please use 'org.apache.spark.sql.avro.functions.to_avro' instead.", "3.0.0")
def to_avro(data: Column): Column = org.apache.spark.sql.avro.functions.to_avro(data)
}
Expand Up @@ -22,7 +22,6 @@ import scala.reflect.ClassTag
import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream
import com.amazonaws.services.kinesis.model.Record

import org.apache.spark.annotation.Evolving
import org.apache.spark.rdd.RDD
import org.apache.spark.storage.{BlockId, StorageLevel}
import org.apache.spark.streaming.{Duration, StreamingContext, Time}
Expand Down Expand Up @@ -84,14 +83,12 @@ private[kinesis] class KinesisInputDStream[T: ClassTag](
}
}

@Evolving
object KinesisInputDStream {
/**
* Builder for [[KinesisInputDStream]] instances.
*
* @since 2.2.0
*/
@Evolving
class Builder {
// Required params
private var streamingContext: Option[StreamingContext] = None
Expand Down
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.streaming.kinesis

import com.amazonaws.auth._

import org.apache.spark.annotation.Evolving
import org.apache.spark.internal.Logging

/**
Expand Down Expand Up @@ -83,14 +82,12 @@ private[kinesis] final case class STSCredentials(
}
}

@Evolving
object SparkAWSCredentials {
/**
* Builder for [[SparkAWSCredentials]] instances.
*
* @since 2.2.0
*/
@Evolving
class Builder {
private var basicCreds: Option[BasicCredentials] = None
private var stsCreds: Option[STSCredentials] = None
Expand Down
Expand Up @@ -24,7 +24,7 @@ import breeze.optimize.{CachedDiffFunction, OWLQN => BreezeOWLQN}
import org.apache.hadoop.fs.Path

import org.apache.spark.SparkException
import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.annotation.Since
import org.apache.spark.internal.Logging
import org.apache.spark.ml.feature.Instance
import org.apache.spark.ml.linalg._
Expand Down Expand Up @@ -59,8 +59,6 @@ private[classification] trait LinearSVCParams extends ClassifierParams with HasR
}

/**
* :: Experimental ::
*
* <a href = "https://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM">
* Linear SVM Classifier</a>
*
Expand All @@ -69,7 +67,6 @@ private[classification] trait LinearSVCParams extends ClassifierParams with HasR
*
*/
@Since("2.2.0")
@Experimental
class LinearSVC @Since("2.2.0") (
@Since("2.2.0") override val uid: String)
extends Classifier[Vector, LinearSVC, LinearSVCModel]
Expand Down Expand Up @@ -290,11 +287,9 @@ object LinearSVC extends DefaultParamsReadable[LinearSVC] {
}

/**
* :: Experimental ::
* Linear SVM Model trained by [[LinearSVC]]
*/
@Since("2.2.0")
@Experimental
class LinearSVCModel private[classification] (
@Since("2.2.0") override val uid: String,
@Since("2.2.0") val coefficients: Vector,
Expand Down
Expand Up @@ -26,7 +26,7 @@ import breeze.optimize.{CachedDiffFunction, LBFGS => BreezeLBFGS, LBFGSB => Bree
import org.apache.hadoop.fs.Path

import org.apache.spark.SparkException
import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.annotation.Since
import org.apache.spark.internal.Logging
import org.apache.spark.ml.feature.Instance
import org.apache.spark.ml.linalg._
Expand Down Expand Up @@ -1349,12 +1349,10 @@ private[ml] class MultiClassSummarizer extends Serializable {
}

/**
* :: Experimental ::
* Abstraction for logistic regression results for a given model.
*
* Currently, the summary ignores the instance weights.
*/
@Experimental
sealed trait LogisticRegressionSummary extends Serializable {

/**
Expand Down Expand Up @@ -1482,12 +1480,10 @@ sealed trait LogisticRegressionSummary extends Serializable {
}

/**
* :: Experimental ::
* Abstraction for multiclass logistic regression training results.
* Currently, the training summary ignores the training weights except
* for the objective trace.
*/
@Experimental
sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary {

/** objective function (scaled loss + regularization) at each iteration. */
Expand All @@ -1501,12 +1497,10 @@ sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary
}

/**
* :: Experimental ::
* Abstraction for binary logistic regression results for a given model.
*
* Currently, the summary ignores the instance weights.
*/
@Experimental
sealed trait BinaryLogisticRegressionSummary extends LogisticRegressionSummary {

private val sparkSession = predictions.sparkSession
Expand Down Expand Up @@ -1590,12 +1584,10 @@ sealed trait BinaryLogisticRegressionSummary extends LogisticRegressionSummary {
}

/**
* :: Experimental ::
* Abstraction for binary logistic regression training results.
* Currently, the training summary ignores the training weights except
* for the objective trace.
*/
@Experimental
sealed trait BinaryLogisticRegressionTrainingSummary extends BinaryLogisticRegressionSummary
with LogisticRegressionTrainingSummary

Expand Down
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.ml.clustering

import org.apache.hadoop.fs.Path

import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.annotation.Since
import org.apache.spark.ml.{Estimator, Model}
import org.apache.spark.ml.linalg.Vector
import org.apache.spark.ml.param._
Expand Down Expand Up @@ -301,7 +301,6 @@ object BisectingKMeans extends DefaultParamsReadable[BisectingKMeans] {


/**
* :: Experimental ::
* Summary of BisectingKMeans.
*
* @param predictions `DataFrame` produced by `BisectingKMeansModel.transform()`.
Expand All @@ -313,7 +312,6 @@ object BisectingKMeans extends DefaultParamsReadable[BisectingKMeans] {
* dataset. This is equivalent to sklearn's inertia.
*/
@Since("2.1.0")
@Experimental
class BisectingKMeansSummary private[clustering] (
predictions: DataFrame,
predictionCol: String,
Expand Down
Expand Up @@ -17,11 +17,10 @@

package org.apache.spark.ml.clustering

import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.annotation.Since
import org.apache.spark.sql.{DataFrame, Row}

/**
* :: Experimental ::
* Summary of clustering algorithms.
*
* @param predictions `DataFrame` produced by model.transform().
Expand All @@ -30,7 +29,6 @@ import org.apache.spark.sql.{DataFrame, Row}
* @param k Number of clusters.
* @param numIter Number of iterations.
*/
@Experimental
class ClusteringSummary private[clustering] (
@transient val predictions: DataFrame,
val predictionCol: String,
Expand Down
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.ml.clustering
import breeze.linalg.{DenseVector => BDV}
import org.apache.hadoop.fs.Path

import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.annotation.Since
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.ml.{Estimator, Model}
import org.apache.spark.ml.impl.Utils.EPSILON
Expand Down Expand Up @@ -697,7 +697,6 @@ private class ExpectationAggregator(
}

/**
* :: Experimental ::
* Summary of GaussianMixture.
*
* @param predictions `DataFrame` produced by `GaussianMixtureModel.transform()`.
Expand All @@ -710,7 +709,6 @@ private class ExpectationAggregator(
* @param numIter Number of iterations.
*/
@Since("2.0.0")
@Experimental
class GaussianMixtureSummary private[clustering] (
predictions: DataFrame,
predictionCol: String,
Expand Down
Expand Up @@ -21,7 +21,7 @@ import scala.collection.mutable

import org.apache.hadoop.fs.Path

import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.annotation.Since
import org.apache.spark.ml.{Estimator, Model, PipelineStage}
import org.apache.spark.ml.linalg.Vector
import org.apache.spark.ml.param._
Expand Down Expand Up @@ -359,7 +359,6 @@ object KMeans extends DefaultParamsReadable[KMeans] {
}

/**
* :: Experimental ::
* Summary of KMeans.
*
* @param predictions `DataFrame` produced by `KMeansModel.transform()`.
Expand All @@ -371,7 +370,6 @@ object KMeans extends DefaultParamsReadable[KMeans] {
* points in the training dataset). This is equivalent to sklearn's inertia.
*/
@Since("2.0.0")
@Experimental
class KMeansSummary private[clustering] (
predictions: DataFrame,
predictionCol: String,
Expand Down
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.ml.clustering

import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.annotation.Since
import org.apache.spark.ml.param._
import org.apache.spark.ml.param.shared._
import org.apache.spark.ml.util._
Expand Down Expand Up @@ -95,7 +95,6 @@ private[clustering] trait PowerIterationClusteringParams extends Params with Has
}

/**
* :: Experimental ::
* Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by
* <a href=http://www.cs.cmu.edu/~frank/papers/icml2010-pic-final.pdf>Lin and Cohen</a>. From
* the abstract: PIC finds a very low-dimensional embedding of a dataset using truncated power
Expand All @@ -108,7 +107,6 @@ private[clustering] trait PowerIterationClusteringParams extends Params with Has
* Spectral clustering (Wikipedia)</a>
*/
@Since("2.4.0")
@Experimental
class PowerIterationClustering private[clustering] (
@Since("2.4.0") override val uid: String)
extends PowerIterationClusteringParams with DefaultParamsWritable {
Expand Down

0 comments on commit eb037a8

Please sign in to comment.