diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java index 7b28a9a96948..a7afbfa8621c 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -33,7 +33,7 @@ public ChunkFetchFailure(StreamChunkId streamChunkId, String errorString) { } @Override - public Type type() { return Type.ChunkFetchFailure; } + public Message.Type type() { return Type.ChunkFetchFailure; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java index 26d063feb5fe..fe54fcc50dc8 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java @@ -32,7 +32,7 @@ public ChunkFetchRequest(StreamChunkId streamChunkId) { } @Override - public Type type() { return Type.ChunkFetchRequest; } + public Message.Type type() { return Type.ChunkFetchRequest; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java index 94c2ac9b20e4..d5c9a9b3202f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java @@ -39,7 +39,7 @@ public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) { } @Override - public Type type() { return Type.ChunkFetchSuccess; } + public Message.Type type() { return Type.ChunkFetchSuccess; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java index f7ffb1bd49bb..1632fb9e0368 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java @@ -34,7 +34,7 @@ public OneWayMessage(ManagedBuffer body) { } @Override - public Type type() { return Type.OneWayMessage; } + public Message.Type type() { return Type.OneWayMessage; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java index a76624ef5dc9..61061903de23 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -31,7 +31,7 @@ public RpcFailure(long requestId, String errorString) { } @Override - public Type type() { return Type.RpcFailure; } + public Message.Type type() { return Type.RpcFailure; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java index 2b30920f0598..cc1bb95d2d56 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -38,7 +38,7 @@ public RpcRequest(long requestId, ManagedBuffer message) { } @Override - public Type type() { return Type.RpcRequest; } + public Message.Type type() { return Type.RpcRequest; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java index d73014ecd850..c03291e9c0b2 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -33,7 +33,7 @@ public RpcResponse(long requestId, ManagedBuffer message) { } @Override - public Type type() { return Type.RpcResponse; } + public Message.Type type() { return Type.RpcResponse; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java index 258ef81c6783..68fcfa774861 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java @@ -33,7 +33,7 @@ public StreamFailure(String streamId, String error) { } @Override - public Type type() { return Type.StreamFailure; } + public Message.Type type() { return Type.StreamFailure; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java index dc183c043ed9..1b135af752bd 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java @@ -34,7 +34,7 @@ public StreamRequest(String streamId) { } @Override - public Type type() { return Type.StreamRequest; } + public Message.Type type() { return Type.StreamRequest; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java index 50b811604b84..568108c4fe5e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java @@ -40,7 +40,7 @@ public StreamResponse(String streamId, long byteCount, ManagedBuffer buffer) { } @Override - public Type type() { return Type.StreamResponse; } + public Message.Type type() { return Type.StreamResponse; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java index fa1d26e76b85..7d21151e0107 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java @@ -52,7 +52,7 @@ private UploadStream(long requestId, ManagedBuffer meta, long bodyByteCount) { } @Override - public Type type() { return Type.UploadStream; } + public Message.Type type() { return Type.UploadStream; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java index 7331c2b481fb..1b03300d948e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java @@ -23,6 +23,7 @@ import org.apache.spark.network.buffer.NettyManagedBuffer; import org.apache.spark.network.protocol.Encoders; import org.apache.spark.network.protocol.AbstractMessage; +import org.apache.spark.network.protocol.Message; /** * Encodes a Sasl-related message which is attempting to authenticate using some credentials tagged @@ -46,7 +47,7 @@ class SaslMessage extends AbstractMessage { } @Override - public Type type() { return Type.User; } + public Message.Type type() { return Type.User; } @Override public int encodedLength() { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index f309dda8afca..6bf3da94030d 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -101,7 +101,7 @@ void createAndStart(String[] blockIds, BlockFetchingListener listener) public RetryingBlockFetcher( TransportConf conf, - BlockFetchStarter fetchStarter, + RetryingBlockFetcher.BlockFetchStarter fetchStarter, String[] blockIds, BlockFetchingListener listener) { this.fetchStarter = fetchStarter; diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Evolving.java b/common/tags/src/main/java/org/apache/spark/annotation/Evolving.java new file mode 100644 index 000000000000..87e8948f204f --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/Evolving.java @@ -0,0 +1,30 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * APIs that are meant to evolve towards becoming stable APIs, but are not stable APIs yet. + * Evolving interfaces can change from one feature release to another release (i.e. 2.1 to 2.2). + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Evolving {} diff --git a/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java b/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java deleted file mode 100644 index 02bcec737e80..000000000000 --- a/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.annotation; - -import java.lang.annotation.*; - -/** - * Annotation to inform users of how much to rely on a particular package, - * class or method not changing over time. - */ -public class InterfaceStability { - - /** - * Stable APIs that retain source and binary compatibility within a major release. - * These interfaces can change from one major release to another major release - * (e.g. from 1.0 to 2.0). - */ - @Documented - @Retention(RetentionPolicy.RUNTIME) - @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, - ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) - public @interface Stable {}; - - /** - * APIs that are meant to evolve towards becoming stable APIs, but are not stable APIs yet. - * Evolving interfaces can change from one feature release to another release (i.e. 2.1 to 2.2). - */ - @Documented - @Retention(RetentionPolicy.RUNTIME) - @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, - ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) - public @interface Evolving {}; - - /** - * Unstable APIs, with no guarantee on stability. - * Classes that are unannotated are considered Unstable. - */ - @Documented - @Retention(RetentionPolicy.RUNTIME) - @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, - ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) - public @interface Unstable {}; -} diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Stable.java b/common/tags/src/main/java/org/apache/spark/annotation/Stable.java new file mode 100644 index 000000000000..b198bfbe91e1 --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/Stable.java @@ -0,0 +1,31 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * Stable APIs that retain source and binary compatibility within a major release. + * These interfaces can change from one major release to another major release + * (e.g. from 1.0 to 2.0). + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Stable {} diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Unstable.java b/common/tags/src/main/java/org/apache/spark/annotation/Unstable.java new file mode 100644 index 000000000000..88ee72125b23 --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/Unstable.java @@ -0,0 +1,30 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * Unstable APIs, with no guarantee on stability. + * Classes that are unannotated are considered Unstable. + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Unstable {} diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala index 1ffec01df9f0..d4a428f45c11 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala @@ -22,7 +22,7 @@ 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.InterfaceStability +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} @@ -84,14 +84,14 @@ private[kinesis] class KinesisInputDStream[T: ClassTag]( } } -@InterfaceStability.Evolving +@Evolving object KinesisInputDStream { /** * Builder for [[KinesisInputDStream]] instances. * * @since 2.2.0 */ - @InterfaceStability.Evolving + @Evolving class Builder { // Required params private var streamingContext: Option[StreamingContext] = None diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala index 9facfe8ff2b0..dcb60b21d985 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala @@ -14,13 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.streaming.kinesis -import scala.collection.JavaConverters._ +package org.apache.spark.streaming.kinesis import com.amazonaws.auth._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging /** @@ -84,14 +83,14 @@ private[kinesis] final case class STSCredentials( } } -@InterfaceStability.Evolving +@Evolving object SparkAWSCredentials { /** * Builder for [[SparkAWSCredentials]] instances. * * @since 2.2.0 */ - @InterfaceStability.Evolving + @Evolving class Builder { private var basicCreds: Option[BasicCredentials] = None private var stsCreds: Option[STSCredentials] = None diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java index 9cbebdaeb33d..0999cbd21687 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractAppHandle.java @@ -31,8 +31,8 @@ abstract class AbstractAppHandle implements SparkAppHandle { private final LauncherServer server; private LauncherServer.ServerConnection connection; - private List listeners; - private AtomicReference state; + private List listeners; + private AtomicReference state; private volatile String appId; private volatile boolean disposed; @@ -42,7 +42,7 @@ protected AbstractAppHandle(LauncherServer server) { } @Override - public synchronized void addListener(Listener l) { + public synchronized void addListener(SparkAppHandle.Listener l) { if (listeners == null) { listeners = new CopyOnWriteArrayList<>(); } @@ -50,7 +50,7 @@ public synchronized void addListener(Listener l) { } @Override - public State getState() { + public SparkAppHandle.State getState() { return state.get(); } @@ -120,11 +120,11 @@ synchronized void dispose() { } } - void setState(State s) { + void setState(SparkAppHandle.State s) { setState(s, false); } - void setState(State s, boolean force) { + void setState(SparkAppHandle.State s, boolean force) { if (force) { state.set(s); fireEvent(false); diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index d985f8ca1ecc..fbc7be25a564 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -31,7 +31,7 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.annotation.{DeveloperApi, InterfaceStability, Since} +import org.apache.spark.annotation.{DeveloperApi, Since, Unstable} import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.classification.{OneVsRest, OneVsRestModel} @@ -84,7 +84,7 @@ private[util] sealed trait BaseReadWrite { * * @since 2.4.0 */ -@InterfaceStability.Unstable +@Unstable @Since("2.4.0") trait MLWriterFormat { /** @@ -108,7 +108,7 @@ trait MLWriterFormat { * * @since 2.4.0 */ -@InterfaceStability.Unstable +@Unstable @Since("2.4.0") trait MLFormatRegister extends MLWriterFormat { /** @@ -208,7 +208,7 @@ abstract class MLWriter extends BaseReadWrite with Logging { /** * A ML Writer which delegates based on the requested format. */ -@InterfaceStability.Unstable +@Unstable @Since("2.4.0") class GeneralMLWriter(stage: PipelineStage) extends MLWriter with Logging { private var source: String = "internal" @@ -291,7 +291,7 @@ trait MLWritable { * Trait for classes that provide `GeneralMLWriter`. */ @Since("2.4.0") -@InterfaceStability.Unstable +@Unstable trait GeneralMLWritable extends MLWritable { /** * Returns an `MLWriter` instance for this ML instance. diff --git a/pom.xml b/pom.xml index 59e3d0fa772b..fcec295eee12 100644 --- a/pom.xml +++ b/pom.xml @@ -2016,7 +2016,6 @@ net.alchim31.maven scala-maven-plugin - 3.4.4 @@ -2037,6 +2036,13 @@ testCompile + + attach-scaladocs + verify + + doc-jar + + ${scala.version} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java index 2ce1fdcbf56a..0258e66ffb6e 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java @@ -17,7 +17,7 @@ package org.apache.spark.sql; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; import org.apache.spark.sql.catalyst.expressions.GenericRow; /** @@ -25,7 +25,7 @@ * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable public class RowFactory { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 1b2f5eee5ccd..5395e4035e68 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -50,7 +50,7 @@ public final class UnsafeExternalRowSorter { private long numRowsInserted = 0; private final StructType schema; - private final PrefixComputer prefixComputer; + private final UnsafeExternalRowSorter.PrefixComputer prefixComputer; private final UnsafeExternalSorter sorter; public abstract static class PrefixComputer { @@ -74,7 +74,7 @@ public static UnsafeExternalRowSorter createWithRecordComparator( StructType schema, Supplier recordComparatorSupplier, PrefixComparator prefixComparator, - PrefixComputer prefixComputer, + UnsafeExternalRowSorter.PrefixComputer prefixComputer, long pageSizeBytes, boolean canUseRadixSort) throws IOException { return new UnsafeExternalRowSorter(schema, recordComparatorSupplier, prefixComparator, @@ -85,7 +85,7 @@ public static UnsafeExternalRowSorter create( StructType schema, Ordering ordering, PrefixComparator prefixComparator, - PrefixComputer prefixComputer, + UnsafeExternalRowSorter.PrefixComputer prefixComputer, long pageSizeBytes, boolean canUseRadixSort) throws IOException { Supplier recordComparatorSupplier = @@ -98,9 +98,9 @@ private UnsafeExternalRowSorter( StructType schema, Supplier recordComparatorSupplier, PrefixComparator prefixComparator, - PrefixComputer prefixComputer, + UnsafeExternalRowSorter.PrefixComputer prefixComputer, long pageSizeBytes, - boolean canUseRadixSort) throws IOException { + boolean canUseRadixSort) { this.schema = schema; this.prefixComputer = prefixComputer; final SparkEnv sparkEnv = SparkEnv.get(); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java index 5f1032d1229d..5f6a46f2b8e8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java @@ -17,8 +17,8 @@ package org.apache.spark.sql.streaming; +import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.catalyst.plans.logical.*; /** @@ -29,7 +29,7 @@ * @since 2.2.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving public class GroupStateTimeout { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java index 470c128ee6c3..a3d72a1f5d49 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.streaming.InternalOutputModes; /** @@ -26,7 +26,7 @@ * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving public class OutputMode { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java index 0f8570fe470b..d786374f69e2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java @@ -19,7 +19,7 @@ import java.util.*; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * To get/create specific data type, users should use singleton objects and factory methods @@ -27,7 +27,7 @@ * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable public class DataTypes { /** * Gets the StringType object. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java index 1290614a3207..a54398324fc6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java @@ -20,7 +20,7 @@ import java.lang.annotation.*; import org.apache.spark.annotation.DeveloperApi; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * ::DeveloperApi:: @@ -31,7 +31,7 @@ @DeveloperApi @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.TYPE) -@InterfaceStability.Evolving +@Evolving public @interface SQLUserDefinedType { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 50ee6cd4085e..f5c87677ab9e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -17,16 +17,15 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - /** * Thrown when a query fails to analyze, usually because the query itself is invalid. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class AnalysisException protected[sql] ( val message: String, val line: Option[Int] = None, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala index 7b02317b8538..9853a4fcc2f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala @@ -20,10 +20,9 @@ package org.apache.spark.sql import scala.annotation.implicitNotFound import scala.reflect.ClassTag -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.types._ - /** * :: Experimental :: * Used to convert a JVM object of type `T` to and from the internal Spark SQL representation. @@ -67,7 +66,7 @@ import org.apache.spark.sql.types._ * @since 1.6.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving @implicitNotFound("Unable to find encoder for type ${T}. An implicit Encoder[${T}] is needed to " + "store ${T} instances in a Dataset. Primitive types (Int, String, etc) and Product types (case " + "classes) are supported by importing spark.implicits._ Support for serializing other types " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index 8a30c81912fe..42b865c02720 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -22,7 +22,7 @@ import java.lang.reflect.Modifier import scala.reflect.{classTag, ClassTag} import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{BoundReference, Cast} @@ -36,7 +36,7 @@ import org.apache.spark.sql.types._ * @since 1.6.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving object Encoders { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 180c2d130074..e12bf9616e2d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.util.hashing.MurmurHash3 -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.types.StructType /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object Row { /** * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: @@ -124,7 +124,7 @@ object Row { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait Row extends Serializable { /** Number of elements in the Row. */ def size: Int = length diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index c43cc748655e..5367ce2af8e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.types import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.Expression /** @@ -134,7 +134,7 @@ object AtomicType { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class NumericType extends AtomicType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 58c75b5dc7a3..7465569868f0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -21,7 +21,7 @@ import scala.math.Ordering import org.json4s.JsonDSL._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.ArrayData /** @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.util.ArrayData * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object ArrayType extends AbstractDataType { /** * Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. @@ -60,7 +60,7 @@ object ArrayType extends AbstractDataType { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { /** No-arg constructor for kryo. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala index 032d6b54aeb7..cc8b3e6e399a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala @@ -20,15 +20,14 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.TypeUtils - /** * The data type representing `Array[Byte]` values. * Please use the singleton `DataTypes.BinaryType`. */ -@InterfaceStability.Stable +@Stable class BinaryType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. @@ -55,5 +54,5 @@ class BinaryType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object BinaryType extends BinaryType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala index 63f354d2243c..5e3de71caa37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala @@ -20,15 +20,14 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability - +import org.apache.spark.annotation.Stable /** * The data type representing `Boolean` values. Please use the singleton `DataTypes.BooleanType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class BooleanType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. @@ -48,5 +47,5 @@ class BooleanType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object BooleanType extends BooleanType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala index 5854c3f5ba11..9d400eefc0f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Byte` values. Please use the singleton `DataTypes.ByteType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class ByteType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. @@ -52,5 +52,5 @@ class ByteType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object ByteType extends ByteType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index 2342036a5746..8e297874a0d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing calendar time intervals. The calendar time interval is stored @@ -29,7 +29,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable class CalendarIntervalType private() extends DataType { override def defaultSize: Int = 16 @@ -40,5 +40,5 @@ class CalendarIntervalType private() extends DataType { /** * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable case object CalendarIntervalType extends CalendarIntervalType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 33fc4b948012..c58f7a239737 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -26,7 +26,7 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -38,7 +38,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class DataType extends AbstractDataType { /** * Enables matching against DataType for expressions: @@ -111,7 +111,7 @@ abstract class DataType extends AbstractDataType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object DataType { private val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala index 9e70dd486a12..7491014b22da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * A date type, supporting "0001-01-01" through "9999-12-31". @@ -31,7 +31,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class DateType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "DateType$" in byte code. @@ -53,5 +53,5 @@ class DateType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object DateType extends DateType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 9eed2eb20204..a3a844670e0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import java.lang.{Long => JLong} import java.math.{BigInteger, MathContext, RoundingMode} -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Unstable import org.apache.spark.sql.AnalysisException /** @@ -31,7 +31,7 @@ import org.apache.spark.sql.AnalysisException * - If decimalVal is set, it represents the whole decimal value * - Otherwise, the decimal value is longVal / (10 ** _scale) */ -@InterfaceStability.Unstable +@Unstable final class Decimal extends Ordered[Decimal] with Serializable { import org.apache.spark.sql.types.Decimal._ @@ -407,7 +407,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } -@InterfaceStability.Unstable +@Unstable object Decimal { val ROUND_HALF_UP = BigDecimal.RoundingMode.HALF_UP val ROUND_HALF_EVEN = BigDecimal.RoundingMode.HALF_EVEN diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 15004e4b9667..25eddaf06a78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -21,11 +21,10 @@ import java.util.Locale import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} - /** * The data type representing `java.math.BigDecimal` values. * A Decimal that must have fixed precision (the maximum number of digits) and scale (the number @@ -39,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class DecimalType(precision: Int, scale: Int) extends FractionalType { if (scale > precision) { @@ -110,7 +109,7 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object DecimalType extends AbstractDataType { import scala.math.min diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala index a5c79ff01ca0..afd335339701 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -21,7 +21,7 @@ import scala.math.{Fractional, Numeric, Ordering} import scala.math.Numeric.DoubleAsIfIntegral import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.util.Utils /** @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class DoubleType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. @@ -54,5 +54,5 @@ class DoubleType private() extends FractionalType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object DoubleType extends DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala index 352147ec936c..6d9898730408 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -21,7 +21,7 @@ import scala.math.{Fractional, Numeric, Ordering} import scala.math.Numeric.FloatAsIfIntegral import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.util.Utils /** @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class FloatType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. @@ -55,5 +55,5 @@ class FloatType private() extends FractionalType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object FloatType extends FloatType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala index a85e3729188d..0755202d20df 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Int` values. Please use the singleton `DataTypes.IntegerType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class IntegerType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. @@ -51,5 +51,5 @@ class IntegerType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object IntegerType extends IntegerType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala index 0997028fc105..3c49c721fdc8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Long` values. Please use the singleton `DataTypes.LongType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class LongType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "LongType$" in byte code. @@ -51,5 +51,5 @@ class LongType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object LongType extends LongType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala index 594e155268bf..29b9ffc0c354 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type for Maps. Keys in a map are not allowed to have `null` values. @@ -31,7 +31,7 @@ import org.apache.spark.annotation.InterfaceStability * @param valueType The data type of map values. * @param valueContainsNull Indicates if map values have `null` values. */ -@InterfaceStability.Stable +@Stable case class MapType( keyType: DataType, valueType: DataType, @@ -78,7 +78,7 @@ case class MapType( /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object MapType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = apply(NullType, NullType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 7c15dc0de4b6..4979aced145c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** @@ -37,7 +37,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable sealed class Metadata private[types] (private[types] val map: Map[String, Any]) extends Serializable { @@ -117,7 +117,7 @@ sealed class Metadata private[types] (private[types] val map: Map[String, Any]) /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object Metadata { private[this] val _empty = new Metadata(Map.empty) @@ -228,7 +228,7 @@ object Metadata { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class MetadataBuilder { private val map: mutable.Map[String, Any] = mutable.Map.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala index 494225b47a27..14097a5280d5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala @@ -17,15 +17,14 @@ package org.apache.spark.sql.types -import org.apache.spark.annotation.InterfaceStability - +import org.apache.spark.annotation.Stable /** * The data type representing `NULL` values. Please use the singleton `DataTypes.NullType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class NullType private() extends DataType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "NullType$" in byte code. @@ -38,5 +37,5 @@ class NullType private() extends DataType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object NullType extends NullType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala index 203e85e1c99b..6756b209f432 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.types import scala.language.existentials -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving -@InterfaceStability.Evolving +@Evolving object ObjectType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = throw new UnsupportedOperationException( @@ -38,7 +38,7 @@ object ObjectType extends AbstractDataType { /** * Represents a JVM object that is passing through Spark SQL expression evaluation. */ -@InterfaceStability.Evolving +@Evolving case class ObjectType(cls: Class[_]) extends DataType { override def defaultSize: Int = 4096 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala index ee655c338b59..9b5ddfef1ccf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Short` values. Please use the singleton `DataTypes.ShortType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class ShortType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. @@ -51,5 +51,5 @@ class ShortType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object ShortType extends ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala index 59b124cda7d1..8ce1cd078e31 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.unsafe.types.UTF8String /** @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.UTF8String * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class StringType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. @@ -48,6 +48,6 @@ class StringType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object StringType extends StringType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 35f9970a0aae..6f6b561d67d4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} /** @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdenti * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class StructField( name: String, dataType: DataType, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 06289b148320..3bef75d5bdb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -24,10 +24,10 @@ import scala.util.control.NonFatal import org.json4s.JsonDSL._ import org.apache.spark.SparkException -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} -import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} +import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.util.Utils /** @@ -95,7 +95,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { /** No-arg constructor for kryo. */ @@ -422,7 +422,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object StructType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = new StructType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala index fdb91e049992..a20f155418f8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `java.sql.Timestamp` values. @@ -28,7 +28,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class TimestampType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. @@ -50,5 +50,5 @@ class TimestampType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object TimestampType extends TimestampType diff --git a/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java b/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java index 802949c0ddb6..d4e1d89491f4 100644 --- a/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java +++ b/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.Iterator; +import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.streaming.GroupState; /** @@ -33,7 +33,7 @@ * @since 2.1.1 */ @Experimental -@InterfaceStability.Evolving +@Evolving public interface FlatMapGroupsWithStateFunction extends Serializable { Iterator call(K key, Iterator values, GroupState state) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java b/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java index 353e9886a8a5..f0abfde843cc 100644 --- a/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java +++ b/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.Iterator; +import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.streaming.GroupState; /** @@ -32,7 +32,7 @@ * @since 2.1.1 */ @Experimental -@InterfaceStability.Evolving +@Evolving public interface MapGroupsWithStateFunction extends Serializable { R call(K key, Iterator values, GroupState state) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java index 1c3c9794fb6b..9cc073f53a3e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java +++ b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java @@ -16,14 +16,14 @@ */ package org.apache.spark.sql; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * SaveMode is used to specify the expected behavior of saving a DataFrame to a data source. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable public enum SaveMode { /** * Append mode means that when saving a DataFrame to a data source, if data/table already exists, diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java index 4eeb7be3f5ab..631d6eb1cfb0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 0 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF0 extends Serializable { R call() throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java index 1460daf27dc2..a5d01406edd8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 1 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF1 extends Serializable { R call(T1 t1) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java index 7c4f1e489708..effe99e30b2a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 10 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF10 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java index 26a05106aebd..e70b18b84b08 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 11 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF11 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java index 8ef7a9904202..339feb34135e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 12 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF12 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java index 5c3b2ec1222e..d346e5c908c6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 13 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF13 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java index 97e744d84346..d27f9f5270f4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 14 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF14 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java index 7ddbf914fc11..b99b57a91d46 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 15 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF15 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java index 0ae5dc7195ad..7899fc4b7ad6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 16 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF16 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java index 03543a556c61..40a7e95724fc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 17 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF17 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java index 46740d344391..47935a935891 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 18 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF18 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java index 33fefd8ecaf1..578b796ff03a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 19 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF19 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java index 9822f19217d7..2f856aa3cf63 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 2 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF2 extends Serializable { R call(T1 t1, T2 t2) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java index 8c5e90182da1..aa8a9fa89704 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 20 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF20 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java index e3b09f5167cf..0fe52bce2eca 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 21 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF21 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java index dc6cfa9097ba..69fd8ca42283 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 22 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF22 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java index 7c264b69ba19..84ffd655672a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 3 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF3 extends Serializable { R call(T1 t1, T2 t2, T3 t3) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java index 58df38fc3c91..dd2dc285c226 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 4 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF4 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java index 4146f96e2eed..795cc21c3f76 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 5 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF5 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java index 25d39654c109..a954684c3c9a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 6 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF6 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java index ce63b6a91adb..03761f2c9ebb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 7 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF7 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java index 0e00209ef6b9..8cd3583b2cbf 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 8 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF8 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java index 077981bb3e3e..78a709779196 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 9 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF9 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java index 82a1169cbe7a..7d1fbe64fc96 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java @@ -17,12 +17,12 @@ package org.apache.spark.sql.execution.datasources; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Unstable; /** * Exception thrown when the parquet reader find column type mismatches. */ -@InterfaceStability.Unstable +@Unstable public class SchemaColumnConvertNotSupportedException extends RuntimeException { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java index ec9c107b1c11..5a72f0c6a255 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java +++ b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java @@ -17,8 +17,8 @@ package org.apache.spark.sql.expressions.javalang; +import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.TypedColumn; import org.apache.spark.sql.execution.aggregate.TypedAverage; @@ -35,7 +35,7 @@ * @since 2.0.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving public class typed { // Note: make sure to keep in sync with typed.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java index f403dc619e86..2a4933d75e8d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; import org.apache.spark.sql.sources.v2.reader.BatchReadSupport; import org.apache.spark.sql.types.StructType; @@ -29,7 +29,7 @@ * This interface is used to create {@link BatchReadSupport} instances when end users run * {@code SparkSession.read.format(...).option(...).load()}. */ -@InterfaceStability.Evolving +@Evolving public interface BatchReadSupportProvider extends DataSourceV2 { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java index bd10c3353bf1..df439e2c02fe 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java @@ -19,7 +19,7 @@ import java.util.Optional; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport; import org.apache.spark.sql.types.StructType; @@ -31,7 +31,7 @@ * This interface is used to create {@link BatchWriteSupport} instances when end users run * {@code Dataset.write.format(...).option(...).save()}. */ -@InterfaceStability.Evolving +@Evolving public interface BatchWriteSupportProvider extends DataSourceV2 { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java index 824c290518ac..b4f2eb34a156 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport; import org.apache.spark.sql.types.StructType; @@ -29,7 +29,7 @@ * This interface is used to create {@link ContinuousReadSupport} instances when end users run * {@code SparkSession.readStream.format(...).option(...).load()} with a continuous trigger. */ -@InterfaceStability.Evolving +@Evolving public interface ContinuousReadSupportProvider extends DataSourceV2 { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java index 83df3be74708..1c5e3a0cd31e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java @@ -26,7 +26,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An immutable string-to-string map in which keys are case-insensitive. This is used to represent @@ -73,7 +73,7 @@ * * */ -@InterfaceStability.Evolving +@Evolving public class DataSourceOptions { private final Map keyLowerCasedMap; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java index 6e31e84bf6c7..eae7a45d1d44 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * The base interface for data source v2. Implementations must have a public, 0-arg constructor. @@ -30,5 +30,5 @@ * If Spark fails to execute any methods in the implementations of this interface (by throwing an * exception), the read action will fail and no Spark job will be submitted. */ -@InterfaceStability.Evolving +@Evolving public interface DataSourceV2 {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java index 61c08e7fa89d..c4d9ef88f607 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport; import org.apache.spark.sql.types.StructType; @@ -29,7 +29,7 @@ * This interface is used to create {@link MicroBatchReadSupport} instances when end users run * {@code SparkSession.readStream.format(...).option(...).load()} with a micro-batch trigger. */ -@InterfaceStability.Evolving +@Evolving public interface MicroBatchReadSupportProvider extends DataSourceV2 { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java index bbe430e29926..c00abd9b685b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java @@ -17,14 +17,14 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to * propagate session configs with the specified key-prefix to all data source operations in this * session. */ -@InterfaceStability.Evolving +@Evolving public interface SessionConfigSupport extends DataSourceV2 { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java index f9ca85d8089b..8ac9c5175086 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.execution.streaming.BaseStreamingSink; import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport; import org.apache.spark.sql.streaming.OutputMode; @@ -30,7 +30,7 @@ * This interface is used to create {@link StreamingWriteSupport} instances when end users run * {@code Dataset.writeStream.format(...).option(...).start()}. */ -@InterfaceStability.Evolving +@Evolving public interface StreamingWriteSupportProvider extends DataSourceV2, BaseStreamingSink { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java index 452ee86675b4..518a8b03a2c6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An interface that defines how to load the data from data source for batch processing. @@ -29,7 +29,7 @@ * {@link ScanConfig}. The {@link ScanConfig} will be used to create input partitions and reader * factory to scan data from the data source with a Spark job. */ -@InterfaceStability.Evolving +@Evolving public interface BatchReadSupport extends ReadSupport { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java index 95c30de907e4..5f5248084bad 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java @@ -19,7 +19,7 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A serializable representation of an input partition returned by @@ -32,7 +32,7 @@ * the actual reading. So {@link InputPartition} must be serializable while {@link PartitionReader} * doesn't need to be. */ -@InterfaceStability.Evolving +@Evolving public interface InputPartition extends Serializable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java index 04ff8d0a19fc..294592595953 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java @@ -20,7 +20,7 @@ import java.io.Closeable; import java.io.IOException; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A partition reader returned by {@link PartitionReaderFactory#createReader(InputPartition)} or @@ -32,7 +32,7 @@ * data sources(whose {@link PartitionReaderFactory#supportColumnarReads(InputPartition)} * returns true). */ -@InterfaceStability.Evolving +@Evolving public interface PartitionReader extends Closeable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java index f35de9310eee..97f4a473953f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java @@ -19,7 +19,7 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -30,7 +30,7 @@ * {@link PartitionReader} (by throwing an exception), corresponding Spark task would fail and * get retried until hitting the maximum retry times. */ -@InterfaceStability.Evolving +@Evolving public interface PartitionReaderFactory extends Serializable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java index a58ddb288f1e..b1f610a82e8a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; /** @@ -27,7 +27,7 @@ * If Spark fails to execute any methods in the implementations of this interface (by throwing an * exception), the read action will fail and no Spark job will be submitted. */ -@InterfaceStability.Evolving +@Evolving public interface ReadSupport { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java index 7462ce282058..a69872a52774 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; /** @@ -31,7 +31,7 @@ * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}, implementations mostly need to * cast the input {@link ScanConfig} to the concrete {@link ScanConfig} class of the data source. */ -@InterfaceStability.Evolving +@Evolving public interface ScanConfig { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java index 4c0eedfddfe2..4922962f7065 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java @@ -17,14 +17,14 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An interface for building the {@link ScanConfig}. Implementations can mixin those * SupportsPushDownXYZ interfaces to do operator pushdown, and keep the operator pushdown result in * the returned {@link ScanConfig}. */ -@InterfaceStability.Evolving +@Evolving public interface ScanConfigBuilder { ScanConfig build(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java index 44799c7d4913..14776f37fed4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java @@ -19,13 +19,13 @@ import java.util.OptionalLong; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An interface to represent statistics for a data source, which is returned by * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}. */ -@InterfaceStability.Evolving +@Evolving public interface Statistics { OptionalLong sizeInBytes(); OptionalLong numRows(); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java index 5e7985f645a0..3a89baa1b44c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java @@ -17,14 +17,14 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.Filter; /** * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this interface to * push down filters to the data source and reduce the size of the data to be read. */ -@InterfaceStability.Evolving +@Evolving public interface SupportsPushDownFilters extends ScanConfigBuilder { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java index edb164937d6e..193476322488 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; /** @@ -25,7 +25,7 @@ * interface to push down required columns to the data source and only read these columns during * scan to reduce the size of the data to be read. */ -@InterfaceStability.Evolving +@Evolving public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java index db62cd451536..0335c7775c2a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; /** @@ -27,7 +27,7 @@ * Note that, when a {@link ReadSupport} implementation creates exactly one {@link InputPartition}, * Spark may avoid adding a shuffle even if the reader does not implement this interface. */ -@InterfaceStability.Evolving +@Evolving public interface SupportsReportPartitioning extends ReadSupport { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java index 1831488ba096..917372cdd25b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to @@ -27,7 +27,7 @@ * data source. Implementations that return more accurate statistics based on pushed operators will * not improve query performance until the planner can push operators before getting stats. */ -@InterfaceStability.Evolving +@Evolving public interface SupportsReportStatistics extends ReadSupport { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java index 6764d4b7665c..1cdc02f5736b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.PartitionReader; /** @@ -25,7 +25,7 @@ * share the same values for the {@link #clusteredColumns} will be produced by the same * {@link PartitionReader}. */ -@InterfaceStability.Evolving +@Evolving public class ClusteredDistribution implements Distribution { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java index 364a3f553923..02b0e6897491 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.PartitionReader; /** @@ -37,5 +37,5 @@ *
  • {@link ClusteredDistribution}
  • * */ -@InterfaceStability.Evolving +@Evolving public interface Distribution {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java index fb0b6f1df43b..c9a00262c128 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.InputPartition; import org.apache.spark.sql.sources.v2.reader.ScanConfig; import org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning; @@ -28,7 +28,7 @@ * like a snapshot. Once created, it should be deterministic and always report the same number of * partitions and the same "satisfy" result for a certain distribution. */ -@InterfaceStability.Evolving +@Evolving public interface Partitioning { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java index 9101c8a44d34..c7f6fce6e81a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java @@ -17,13 +17,13 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.PartitionReader; /** * A variation on {@link PartitionReader} for use with continuous streaming processing. */ -@InterfaceStability.Evolving +@Evolving public interface ContinuousPartitionReader extends PartitionReader { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java index 2d9f1ca1686a..41195befe5e5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.reader.InputPartition; import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; @@ -28,7 +28,7 @@ * instead of {@link org.apache.spark.sql.sources.v2.reader.PartitionReader}. It's used for * continuous streaming processing. */ -@InterfaceStability.Evolving +@Evolving public interface ContinuousPartitionReaderFactory extends PartitionReaderFactory { @Override ContinuousPartitionReader createReader(InputPartition partition); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java index 9a3ad2eb8a80..2b784ac0e9f3 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.execution.streaming.BaseStreamingSource; import org.apache.spark.sql.sources.v2.reader.InputPartition; import org.apache.spark.sql.sources.v2.reader.ScanConfig; @@ -36,7 +36,7 @@ * {@link #stop()} will be called when the streaming execution is completed. Note that a single * query may have multiple executions due to restart or failure recovery. */ -@InterfaceStability.Evolving +@Evolving public interface ContinuousReadSupport extends StreamingReadSupport, BaseStreamingSource { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java index edb0db11bff2..f56066c63938 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.execution.streaming.BaseStreamingSource; import org.apache.spark.sql.sources.v2.reader.*; @@ -33,7 +33,7 @@ * will be called when the streaming execution is completed. Note that a single query may have * multiple executions due to restart or failure recovery. */ -@InterfaceStability.Evolving +@Evolving public interface MicroBatchReadSupport extends StreamingReadSupport, BaseStreamingSource { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java index 6cf27734867c..6104175d2c9e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An abstract representation of progress through a {@link MicroBatchReadSupport} or @@ -30,7 +30,7 @@ * maintain compatibility with DataSource V1 APIs. This extension will be removed once we * get rid of V1 completely. */ -@InterfaceStability.Evolving +@Evolving public abstract class Offset extends org.apache.spark.sql.execution.streaming.Offset { /** * A JSON-serialized representation of an Offset that is diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java index 383e73db6762..2c97d924a062 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java @@ -19,7 +19,7 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * Used for per-partition offsets in continuous processing. ContinuousReader implementations will @@ -27,6 +27,6 @@ * * These offsets must be serializable. */ -@InterfaceStability.Evolving +@Evolving public interface PartitionOffset extends Serializable { } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java index 0ec9e05d6a02..efe1ac4f78db 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.writer; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An interface that defines how to write the data to data source for batch processing. @@ -37,7 +37,7 @@ * * Please refer to the documentation of commit/abort methods for detailed specifications. */ -@InterfaceStability.Evolving +@Evolving public interface BatchWriteSupport { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java index 5fb067966ee6..d142ee523ef9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java @@ -19,7 +19,7 @@ import java.io.IOException; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A data writer returned by {@link DataWriterFactory#createWriter(int, long)} and is @@ -55,7 +55,7 @@ * * Note that, Currently the type `T` can only be {@link org.apache.spark.sql.catalyst.InternalRow}. */ -@InterfaceStability.Evolving +@Evolving public interface DataWriter { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java index 19a36dd23245..65105f46b82d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java @@ -20,7 +20,7 @@ import java.io.Serializable; import org.apache.spark.TaskContext; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; /** @@ -31,7 +31,7 @@ * will be created on executors and do the actual writing. So this interface must be * serializable and {@link DataWriter} doesn't need to be. */ -@InterfaceStability.Evolving +@Evolving public interface DataWriterFactory extends Serializable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java index 123335c414e9..9216e3439909 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java @@ -19,8 +19,8 @@ import java.io.Serializable; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport; -import org.apache.spark.annotation.InterfaceStability; /** * A commit message returned by {@link DataWriter#commit()} and will be sent back to the driver side @@ -30,5 +30,5 @@ * This is an empty interface, data sources should define their own message class and use it when * generating messages at executor side and handling the messages at driver side. */ -@InterfaceStability.Evolving +@Evolving public interface WriterCommitMessage extends Serializable {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java index a4da24fc5ae6..7d3d21cb2b63 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java @@ -20,7 +20,7 @@ import java.io.Serializable; import org.apache.spark.TaskContext; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; @@ -33,7 +33,7 @@ * will be created on executors and do the actual writing. So this interface must be * serializable and {@link DataWriter} doesn't need to be. */ -@InterfaceStability.Evolving +@Evolving public interface StreamingDataWriterFactory extends Serializable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java index 3fdfac5e1c84..84cfbf2dda48 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.writer.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.writer.DataWriter; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; @@ -27,7 +27,7 @@ * Streaming queries are divided into intervals of data called epochs, with a monotonically * increasing numeric ID. This writer handles commits and aborts for each successive epoch. */ -@InterfaceStability.Evolving +@Evolving public interface StreamingWriteSupport { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java index 5371a23230c9..fd6f7be2abc5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java @@ -19,9 +19,9 @@ import java.util.concurrent.TimeUnit; +import org.apache.spark.annotation.Evolving; import scala.concurrent.duration.Duration; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger; import org.apache.spark.sql.execution.streaming.OneTimeTrigger$; @@ -30,7 +30,7 @@ * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving public class Trigger { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 5f58b031f6ae..906e9bc26ef5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -22,7 +22,7 @@ import org.apache.arrow.vector.complex.*; import org.apache.arrow.vector.holders.NullableVarCharHolder; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.execution.arrow.ArrowUtils; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.types.UTF8String; @@ -31,7 +31,7 @@ * A column vector backed by Apache Arrow. Currently calendar interval type and map type are not * supported. */ -@InterfaceStability.Evolving +@Evolving public final class ArrowColumnVector extends ColumnVector { private final ArrowVectorAccessor accessor; diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java index ad99b450a480..14caaeaedbe2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.vectorized; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.CalendarInterval; @@ -47,7 +47,7 @@ * format. Since it is expected to reuse the ColumnVector instance while loading data, the storage * footprint is negligible. */ -@InterfaceStability.Evolving +@Evolving public abstract class ColumnVector implements AutoCloseable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java index 72a192d089b9..dd2bd789c26d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.vectorized; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.types.CalendarInterval; @@ -25,7 +25,7 @@ /** * Array abstraction in {@link ColumnVector}. */ -@InterfaceStability.Evolving +@Evolving public final class ColumnarArray extends ArrayData { // The data for this array. This array contains elements from // data[offset] to data[offset + length). diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java index d206c1df42ab..07546a54013e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java @@ -18,7 +18,7 @@ import java.util.*; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.execution.vectorized.MutableColumnarRow; @@ -27,7 +27,7 @@ * batch so that Spark can access the data row by row. Instance of it is meant to be reused during * the entire data loading process. */ -@InterfaceStability.Evolving +@Evolving public final class ColumnarBatch { private int numRows; private final ColumnVector[] columns; diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java index f2f227959002..4b9d3c5f5991 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.vectorized; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.types.*; @@ -26,7 +26,7 @@ /** * Row abstraction in {@link ColumnVector}. */ -@InterfaceStability.Evolving +@Evolving public final class ColumnarRow extends InternalRow { // The data for this row. // E.g. the value of 3rd int field is `data.getChild(3).getInt(rowId)`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 49d2a34080b1..5a408b29f933 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.language.implicitConversions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} @@ -60,7 +60,7 @@ private[sql] object Column { * * @since 1.6.0 */ -@InterfaceStability.Stable +@Stable class TypedColumn[-T, U]( expr: Expression, private[sql] val encoder: ExpressionEncoder[U]) @@ -130,7 +130,7 @@ class TypedColumn[-T, U]( * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class Column(val expr: Expression) extends Logging { def this(name: String) = this(name match { @@ -1227,7 +1227,7 @@ class Column(val expr: Expression) extends Logging { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class ColumnName(name: String) extends Column(name) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 5288907b7d7f..53e9f810d7c8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -22,18 +22,17 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ - /** * Functionality for working with missing data in `DataFrame`s. * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable final class DataFrameNaFunctions private[sql](df: DataFrame) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index df18623e42a0..52df13d39caa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.Partition -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -48,7 +48,7 @@ import org.apache.spark.unsafe.types.UTF8String * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index 7c12432d33c3..b2f6a6ba8310 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -21,7 +21,7 @@ import java.{lang => jl, util => ju} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.stat._ import org.apache.spark.sql.functions.col @@ -33,7 +33,7 @@ import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch} * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable final class DataFrameStatFunctions private[sql](df: DataFrame) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 1b4998f94b25..29d479f54211 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -21,7 +21,7 @@ import java.util.{Locale, Properties, UUID} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ @@ -40,7 +40,7 @@ import org.apache.spark.sql.types.StructType * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private val df = ds.toDF() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index f98eaa3d4eb9..f5caaf3f7fc8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -21,19 +21,17 @@ import java.io.CharArrayWriter import scala.collection.JavaConverters._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.commons.lang3.StringUtils import org.apache.spark.TaskContext -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Evolving, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.function._ import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.encoders._ @@ -166,10 +164,10 @@ private[sql] object Dataset { * * @since 1.6.0 */ -@InterfaceStability.Stable +@Stable class Dataset[T] private[sql]( @transient val sparkSession: SparkSession, - @DeveloperApi @InterfaceStability.Unstable @transient val queryExecution: QueryExecution, + @DeveloperApi @Unstable @transient val queryExecution: QueryExecution, encoder: Encoder[T]) extends Serializable { @@ -426,7 +424,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def as[U : Encoder]: Dataset[U] = Dataset[U](sparkSession, logicalPlan) /** @@ -544,7 +542,7 @@ class Dataset[T] private[sql]( * @group streaming * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def isStreaming: Boolean = logicalPlan.isStreaming /** @@ -557,7 +555,7 @@ class Dataset[T] private[sql]( * @since 2.1.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def checkpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = true) /** @@ -570,7 +568,7 @@ class Dataset[T] private[sql]( * @since 2.1.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def checkpoint(eager: Boolean): Dataset[T] = checkpoint(eager = eager, reliableCheckpoint = true) /** @@ -583,7 +581,7 @@ class Dataset[T] private[sql]( * @since 2.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def localCheckpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = false) /** @@ -596,7 +594,7 @@ class Dataset[T] private[sql]( * @since 2.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def localCheckpoint(eager: Boolean): Dataset[T] = checkpoint( eager = eager, reliableCheckpoint = false @@ -671,7 +669,7 @@ class Dataset[T] private[sql]( * @group streaming * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving // We only accept an existing column name, not a derived column here as a watermark that is // defined on a derived column cannot referenced elsewhere in the plan. def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { @@ -1066,7 +1064,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def joinWith[U](other: Dataset[U], condition: Column, joinType: String): Dataset[(T, U)] = { // Creates a Join node and resolve it first, to get join condition resolved, self-join resolved, // etc. @@ -1142,7 +1140,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def joinWith[U](other: Dataset[U], condition: Column): Dataset[(T, U)] = { joinWith(other, condition, "inner") } @@ -1384,7 +1382,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1](c1: TypedColumn[T, U1]): Dataset[U1] = { implicit val encoder = c1.encoder val project = Project(c1.withInputType(exprEnc, logicalPlan.output).named :: Nil, logicalPlan) @@ -1418,7 +1416,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2](c1: TypedColumn[T, U1], c2: TypedColumn[T, U2]): Dataset[(U1, U2)] = selectUntyped(c1, c2).asInstanceOf[Dataset[(U1, U2)]] @@ -1430,7 +1428,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2, U3]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1445,7 +1443,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2, U3, U4]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1461,7 +1459,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2, U3, U4, U5]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1632,7 +1630,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def reduce(func: (T, T) => T): T = withNewRDDExecutionId { rdd.reduce(func) } @@ -1647,7 +1645,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def reduce(func: ReduceFunction[T]): T = reduce(func.call(_, _)) /** @@ -1659,7 +1657,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def groupByKey[K: Encoder](func: T => K): KeyValueGroupedDataset[K, T] = { val withGroupingKey = AppendColumns(func, logicalPlan) val executed = sparkSession.sessionState.executePlan(withGroupingKey) @@ -1681,7 +1679,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def groupByKey[K](func: MapFunction[T, K], encoder: Encoder[K]): KeyValueGroupedDataset[K, T] = groupByKey(func.call(_))(encoder) @@ -2483,7 +2481,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def filter(func: T => Boolean): Dataset[T] = { withTypedPlan(TypedFilter(func, logicalPlan)) } @@ -2497,7 +2495,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def filter(func: FilterFunction[T]): Dataset[T] = { withTypedPlan(TypedFilter(func, logicalPlan)) } @@ -2511,7 +2509,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def map[U : Encoder](func: T => U): Dataset[U] = withTypedPlan { MapElements[T, U](func, logicalPlan) } @@ -2525,7 +2523,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def map[U](func: MapFunction[T, U], encoder: Encoder[U]): Dataset[U] = { implicit val uEnc = encoder withTypedPlan(MapElements[T, U](func, logicalPlan)) @@ -2540,7 +2538,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapPartitions[U : Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { new Dataset[U]( sparkSession, @@ -2557,7 +2555,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapPartitions[U](f: MapPartitionsFunction[T, U], encoder: Encoder[U]): Dataset[U] = { val func: (Iterator[T]) => Iterator[U] = x => f.call(x.asJava).asScala mapPartitions(func)(encoder) @@ -2588,7 +2586,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMap[U : Encoder](func: T => TraversableOnce[U]): Dataset[U] = mapPartitions(_.flatMap(func)) @@ -2602,7 +2600,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMap[U](f: FlatMapFunction[T, U], encoder: Encoder[U]): Dataset[U] = { val func: (T) => Iterator[U] = x => f.call(x).asScala flatMap(func)(encoder) @@ -3064,7 +3062,7 @@ class Dataset[T] private[sql]( * @group basic * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def writeStream: DataStreamWriter[T] = { if (!isStreaming) { logicalPlan.failAnalysis( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala b/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala index 08aa1bbe78fa..1c4ffefb897e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * A container for a [[Dataset]], used for implicit conversions in Scala. @@ -30,7 +30,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.6.0 */ -@InterfaceStability.Stable +@Stable case class DatasetHolder[T] private[sql](private val ds: Dataset[T]) { // This is declared with parentheses to prevent the Scala compiler from treating diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index bd8dd6ea3fe0..302d38cde143 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.rules.Rule * @since 1.3.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable class ExperimentalMethods private[sql]() { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala index 52b8c839643e..5c0fe798b104 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving /** * The abstract class for writing custom logic to process data generated by a query. @@ -104,7 +104,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving abstract class ForeachWriter[T] extends Serializable { // TODO: Move this to org.apache.spark.sql.util or consolidate this with batch API. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 555bcdffb6ee..7a47242f6938 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.api.java.function._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateStruct} @@ -37,7 +37,7 @@ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout, OutputMode * @since 2.0.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving class KeyValueGroupedDataset[K, V] private[sql]( kEncoder: Encoder[K], vEncoder: Encoder[V], @@ -237,7 +237,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S: Encoder, U: Encoder]( func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = { val flatMapFunc = (key: K, it: Iterator[V], s: GroupState[S]) => Iterator(func(key, it, s)) @@ -272,7 +272,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S: Encoder, U: Encoder]( timeoutConf: GroupStateTimeout)( func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = { @@ -309,7 +309,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S, U]( func: MapGroupsWithStateFunction[K, V, S, U], stateEncoder: Encoder[S], @@ -340,7 +340,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S, U]( func: MapGroupsWithStateFunction[K, V, S, U], stateEncoder: Encoder[S], @@ -371,7 +371,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMapGroupsWithState[S: Encoder, U: Encoder]( outputMode: OutputMode, timeoutConf: GroupStateTimeout)( @@ -413,7 +413,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMapGroupsWithState[S, U]( func: FlatMapGroupsWithStateFunction[K, V, S, U], outputMode: OutputMode, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index d4e75b5ebd40..e85636d82a62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -22,7 +22,7 @@ import java.util.Locale import scala.collection.JavaConverters._ import scala.language.implicitConversions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} @@ -45,7 +45,7 @@ import org.apache.spark.sql.types.{NumericType, StructType} * * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class RelationalGroupedDataset protected[sql]( df: DataFrame, groupingExprs: Seq[Expression], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 3c39579149ff..5a554eff02e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} import org.apache.spark.sql.internal.SQLConf - /** * Runtime configuration interface for Spark. To access this, use `SparkSession.conf`. * @@ -29,7 +28,7 @@ import org.apache.spark.sql.internal.SQLConf * * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 9982b60fefe6..43f34e6ff4b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -23,7 +23,7 @@ import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation._ import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry @@ -54,7 +54,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager * @groupname Ungrouped Support functions for language integrated queries * @since 1.0.0 */ -@InterfaceStability.Stable +@Stable class SQLContext private[sql](val sparkSession: SparkSession) extends Logging with Serializable { @@ -86,7 +86,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * that listen for execution metrics. */ @Experimental - @InterfaceStability.Evolving + @Evolving def listenerManager: ExecutionListenerManager = sparkSession.listenerManager /** @@ -158,7 +158,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) */ @Experimental @transient - @InterfaceStability.Unstable + @Unstable def experimental: ExperimentalMethods = sparkSession.experimental /** @@ -244,7 +244,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving object implicits extends SQLImplicits with Serializable { protected override def _sqlContext: SQLContext = self } @@ -258,7 +258,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { sparkSession.createDataFrame(rdd) } @@ -271,7 +271,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { sparkSession.createDataFrame(data) } @@ -319,7 +319,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rowRDD, schema) } @@ -363,7 +363,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataset */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { sparkSession.createDataset(data) } @@ -401,7 +401,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataset */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { sparkSession.createDataset(data) } @@ -428,7 +428,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rowRDD, schema) } @@ -443,7 +443,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.6.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rows, schema) } @@ -507,7 +507,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def readStream: DataStreamReader = sparkSession.readStream @@ -631,7 +631,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(end: Long): DataFrame = sparkSession.range(end).toDF() /** @@ -643,7 +643,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long): DataFrame = sparkSession.range(start, end).toDF() /** @@ -655,7 +655,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long): DataFrame = { sparkSession.range(start, end, step).toDF() } @@ -670,7 +670,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long, numPartitions: Int): DataFrame = { sparkSession.range(start, end, step, numPartitions).toDF() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 05db292bd41b..d329af0145c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -21,7 +21,7 @@ import scala.collection.Map import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder * * @since 1.6.0 */ -@InterfaceStability.Evolving +@Evolving abstract class SQLImplicits extends LowPrioritySQLImplicits { protected def _sqlContext: SQLContext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index c0727e844a1c..725db97df4ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -25,7 +25,7 @@ import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Evolving, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -73,7 +73,7 @@ import org.apache.spark.util.{CallSite, Utils} * @param parentSessionState If supplied, inherit all session state (i.e. temporary * views, SQL config, UDFs etc) from parent. */ -@InterfaceStability.Stable +@Stable class SparkSession private( @transient val sparkContext: SparkContext, @transient private val existingSharedState: Option[SharedState], @@ -124,7 +124,7 @@ class SparkSession private( * * @since 2.2.0 */ - @InterfaceStability.Unstable + @Unstable @transient lazy val sharedState: SharedState = { existingSharedState.getOrElse(new SharedState(sparkContext)) @@ -145,7 +145,7 @@ class SparkSession private( * * @since 2.2.0 */ - @InterfaceStability.Unstable + @Unstable @transient lazy val sessionState: SessionState = { parentSessionState @@ -186,7 +186,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def listenerManager: ExecutionListenerManager = sessionState.listenerManager /** @@ -197,7 +197,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Unstable + @Unstable def experimental: ExperimentalMethods = sessionState.experimentalMethods /** @@ -231,7 +231,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Unstable + @Unstable def streams: StreamingQueryManager = sessionState.streamingQueryManager /** @@ -289,7 +289,7 @@ class SparkSession private( * @return 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def emptyDataset[T: Encoder]: Dataset[T] = { val encoder = implicitly[Encoder[T]] new Dataset(self, LocalRelation(encoder.schema.toAttributes), encoder) @@ -302,7 +302,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { SparkSession.setActiveSession(this) val encoder = Encoders.product[A] @@ -316,7 +316,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { SparkSession.setActiveSession(this) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] @@ -356,7 +356,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD, schema, needsConversion = true) } @@ -370,7 +370,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD.rdd, schema) } @@ -384,7 +384,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { Dataset.ofRows(self, LocalRelation.fromExternalRows(schema.toAttributes, rows.asScala)) } @@ -474,7 +474,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -493,7 +493,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { Dataset[T](self, ExternalRDD(data, self)) } @@ -515,7 +515,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } @@ -528,7 +528,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(end: Long): Dataset[java.lang.Long] = range(0, end) /** @@ -539,7 +539,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long): Dataset[java.lang.Long] = { range(start, end, step = 1, numPartitions = sparkContext.defaultParallelism) } @@ -552,7 +552,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long): Dataset[java.lang.Long] = { range(start, end, step, numPartitions = sparkContext.defaultParallelism) } @@ -566,7 +566,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[java.lang.Long] = { new Dataset(self, Range(start, end, step, numPartitions), Encoders.LONG) } @@ -672,7 +672,7 @@ class SparkSession private( * * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def readStream: DataStreamReader = new DataStreamReader(self) /** @@ -706,7 +706,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving object implicits extends SQLImplicits with Serializable { protected override def _sqlContext: SQLContext = SparkSession.this.sqlContext } @@ -775,13 +775,13 @@ class SparkSession private( } -@InterfaceStability.Stable +@Stable object SparkSession extends Logging { /** * Builder for [[SparkSession]]. */ - @InterfaceStability.Stable + @Stable class Builder extends Logging { private[this] val options = new scala.collection.mutable.HashMap[String, String] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index a4864344b2d2..5ed76789786b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import scala.collection.mutable -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Unstable} import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -66,7 +66,7 @@ import org.apache.spark.sql.catalyst.rules.Rule */ @DeveloperApi @Experimental -@InterfaceStability.Unstable +@Unstable class SparkSessionExtensions { type RuleBuilder = SparkSession => Rule[LogicalPlan] type CheckRuleBuilder = SparkSession => LogicalPlan => Unit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index 84da097be53c..5a3f556c9c07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -22,7 +22,7 @@ import java.lang.reflect.ParameterizedType import scala.reflect.runtime.universe.TypeTag import scala.util.Try -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.internal.Logging import org.apache.spark.sql.api.java._ @@ -44,7 +44,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends Logging { protected[sql] def registerPython(name: String, udf: UserDefinedPythonFunction): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala index ab81725def3f..44668610d805 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalog import scala.collection.JavaConverters._ -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental, Stable} import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset} import org.apache.spark.sql.types.StructType import org.apache.spark.storage.StorageLevel @@ -29,7 +29,7 @@ import org.apache.spark.storage.StorageLevel * * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable abstract class Catalog { /** @@ -233,7 +233,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable(tableName: String, path: String): DataFrame /** @@ -261,7 +261,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable(tableName: String, path: String, source: String): DataFrame /** @@ -292,7 +292,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, @@ -330,7 +330,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, @@ -366,7 +366,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, @@ -406,7 +406,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala index c0c5ebc2ba2d..cb270875228a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalog import javax.annotation.Nullable -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.DefinedByConstructorParams @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams * @param locationUri path (in the form of a uri) to data files. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Database( val name: String, @Nullable val description: String, @@ -61,7 +61,7 @@ class Database( * @param isTemporary whether the table is a temporary table. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Table( val name: String, @Nullable val database: String, @@ -93,7 +93,7 @@ class Table( * @param isBucket whether the column is a bucket column. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Column( val name: String, @Nullable val description: String, @@ -126,7 +126,7 @@ class Column( * @param isTemporary whether the function is a temporary function or not. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Function( val name: String, @Nullable val database: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 19e3e55cb282..4c0db3cb42a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.streaming.Trigger /** @@ -25,5 +25,5 @@ import org.apache.spark.sql.streaming.Trigger * the query. */ @Experimental -@InterfaceStability.Evolving +@Evolving case object OneTimeTrigger extends Trigger diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala index 90e1766c4d9f..caffcc3c4c1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala @@ -23,15 +23,15 @@ import scala.concurrent.duration.Duration import org.apache.commons.lang3.StringUtils -import org.apache.spark.annotation.{Experimental, InterfaceStability} -import org.apache.spark.sql.streaming.{ProcessingTime, Trigger} +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.streaming.Trigger import org.apache.spark.unsafe.types.CalendarInterval /** * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at * the specified interval. */ -@InterfaceStability.Evolving +@Evolving case class ContinuousTrigger(intervalMs: Long) extends Trigger { require(intervalMs >= 0, "the interval of trigger should not be negative") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala index 1e076207bc60..6b4def35e195 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.{Experimental, InterfaceStability} -import org.apache.spark.sql.{Dataset, Encoder, TypedColumn} +import org.apache.spark.annotation.{Evolving, Experimental} +import org.apache.spark.sql.{Encoder, TypedColumn} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression @@ -51,7 +51,7 @@ import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression * @since 1.6.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving abstract class Aggregator[-IN, BUF, OUT] extends Serializable { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index eb956c4b3e88..58a942afe28c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.ScalaUDF @@ -37,7 +37,7 @@ import org.apache.spark.sql.types.DataType * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class UserDefinedFunction protected[sql] ( f: AnyRef, dataType: DataType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index d50031bb2062..3d8d931af218 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions._ @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.expressions._ * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable object Window { /** @@ -234,5 +234,5 @@ object Window { * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable class Window private() // So we can see Window in JavaDoc. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index b7f3000880ac..58227f075f2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.{AnalysisException, Column} import org.apache.spark.sql.catalyst.expressions._ @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable class WindowSpec private[sql]( partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala index 3e637d594caf..1cb579c4faa7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions.scalalang -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql._ import org.apache.spark.sql.execution.aggregate._ @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.aggregate._ * @since 2.0.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving // scalastyle:off object typed { // scalastyle:on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index 4976b875fa29..4e8cb3a6ddd6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.{Column, Row} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.ScalaUDAF @@ -28,7 +28,7 @@ import org.apache.spark.sql.types._ * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable abstract class UserDefinedAggregateFunction extends Serializable { /** @@ -159,7 +159,7 @@ abstract class UserDefinedAggregateFunction extends Serializable { * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable abstract class MutableAggregationBuffer extends Row { /** Update the ith value of this buffer. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index b2a6e22cbfc8..1cf2a30c0c8b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -23,7 +23,7 @@ import scala.reflect.runtime.universe.{typeTag, TypeTag} import scala.util.Try import scala.util.control.NonFatal -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedFunction} @@ -68,7 +68,7 @@ import org.apache.spark.util.Utils * @groupname Ungrouped Support functions for DataFrames * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable // scalastyle:off object functions { // scalastyle:on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index f67cc32c15dd..ac07e1f6bb4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal import org.apache.spark.SparkConf -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql.{ExperimentalMethods, SparkSession, UDFRegistration, _} import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog.SessionCatalog @@ -50,7 +50,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager * and `catalog` fields. Note that the state is cloned when `build` is called, and not before. */ @Experimental -@InterfaceStability.Unstable +@Unstable abstract class BaseSessionStateBuilder( val session: SparkSession, val parentState: Option[SessionState] = None) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index accbea41b960..b34db581ca2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog._ @@ -124,7 +124,7 @@ private[sql] object SessionState { * Concrete implementation of a [[BaseSessionStateBuilder]]. */ @Experimental -@InterfaceStability.Unstable +@Unstable class SessionStateBuilder( session: SparkSession, parentState: Option[SessionState] = None) @@ -135,7 +135,7 @@ class SessionStateBuilder( /** * Session shared [[FunctionResourceLoader]]. */ -@InterfaceStability.Unstable +@Unstable class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoader { override def loadResource(resource: FunctionResource): Unit = { resource.resourceType match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index f76c1fae562c..230b43022b02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -21,8 +21,7 @@ import java.sql.{Connection, Date, Timestamp} import org.apache.commons.lang3.StringUtils -import org.apache.spark.annotation.{DeveloperApi, InterfaceStability, Since} -import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.annotation.{DeveloperApi, Evolving, Since} import org.apache.spark.sql.types._ /** @@ -34,7 +33,7 @@ import org.apache.spark.sql.types._ * send a null value to the database. */ @DeveloperApi -@InterfaceStability.Evolving +@Evolving case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) /** @@ -57,7 +56,7 @@ case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) * for the given Catalyst type. */ @DeveloperApi -@InterfaceStability.Evolving +@Evolving abstract class JdbcDialect extends Serializable { /** * Check if this dialect instance can handle a certain jdbc url. @@ -197,7 +196,7 @@ abstract class JdbcDialect extends Serializable { * sure to register your dialects first. */ @DeveloperApi -@InterfaceStability.Evolving +@Evolving object JdbcDialects { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 354660e9d594..61875931d226 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.annotation.{DeveloperApi, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Unstable} import org.apache.spark.sql.execution.SparkStrategy /** @@ -40,7 +40,7 @@ package object sql { * [[org.apache.spark.sql.sources]] */ @DeveloperApi - @InterfaceStability.Unstable + @Unstable type Strategy = SparkStrategy type DataFrame = Dataset[Row] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala index bdd8c4da6bd3..3f941cc6e107 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines all the filters that we can push down to the data sources. @@ -28,7 +28,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class Filter { /** * List of columns that are referenced by this filter. @@ -48,7 +48,7 @@ abstract class Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class EqualTo(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -60,7 +60,7 @@ case class EqualTo(attribute: String, value: Any) extends Filter { * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable case class EqualNullSafe(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -71,7 +71,7 @@ case class EqualNullSafe(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class GreaterThan(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -82,7 +82,7 @@ case class GreaterThan(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -93,7 +93,7 @@ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class LessThan(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -104,7 +104,7 @@ case class LessThan(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class LessThanOrEqual(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -114,7 +114,7 @@ case class LessThanOrEqual(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class In(attribute: String, values: Array[Any]) extends Filter { override def hashCode(): Int = { var h = attribute.hashCode @@ -141,7 +141,7 @@ case class In(attribute: String, values: Array[Any]) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class IsNull(attribute: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -151,7 +151,7 @@ case class IsNull(attribute: String) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class IsNotNull(attribute: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -161,7 +161,7 @@ case class IsNotNull(attribute: String) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class And(left: Filter, right: Filter) extends Filter { override def references: Array[String] = left.references ++ right.references } @@ -171,7 +171,7 @@ case class And(left: Filter, right: Filter) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class Or(left: Filter, right: Filter) extends Filter { override def references: Array[String] = left.references ++ right.references } @@ -181,7 +181,7 @@ case class Or(left: Filter, right: Filter) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class Not(child: Filter) extends Filter { override def references: Array[String] = child.references } @@ -192,7 +192,7 @@ case class Not(child: Filter) extends Filter { * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable case class StringStartsWith(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -203,7 +203,7 @@ case class StringStartsWith(attribute: String, value: String) extends Filter { * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable case class StringEndsWith(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -214,7 +214,7 @@ case class StringEndsWith(attribute: String, value: String) extends Filter { * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable case class StringContains(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 6057a795c8bf..6ad054c9f640 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation._ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -35,7 +35,7 @@ import org.apache.spark.sql.types.StructType * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable trait DataSourceRegister { /** @@ -65,7 +65,7 @@ trait DataSourceRegister { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait RelationProvider { /** * Returns a new base relation with the given parameters. @@ -96,7 +96,7 @@ trait RelationProvider { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait SchemaRelationProvider { /** * Returns a new base relation with the given parameters and user defined schema. @@ -117,7 +117,7 @@ trait SchemaRelationProvider { * @since 2.0.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable trait StreamSourceProvider { /** @@ -148,7 +148,7 @@ trait StreamSourceProvider { * @since 2.0.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable trait StreamSinkProvider { def createSink( sqlContext: SQLContext, @@ -160,7 +160,7 @@ trait StreamSinkProvider { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait CreatableRelationProvider { /** * Saves a DataFrame to a destination (using data source-specific parameters) @@ -192,7 +192,7 @@ trait CreatableRelationProvider { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class BaseRelation { def sqlContext: SQLContext def schema: StructType @@ -242,7 +242,7 @@ abstract class BaseRelation { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait TableScan { def buildScan(): RDD[Row] } @@ -253,7 +253,7 @@ trait TableScan { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait PrunedScan { def buildScan(requiredColumns: Array[String]): RDD[Row] } @@ -271,7 +271,7 @@ trait PrunedScan { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait PrunedFilteredScan { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } @@ -293,7 +293,7 @@ trait PrunedFilteredScan { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit } @@ -309,7 +309,7 @@ trait InsertableRelation { * @since 1.3.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable trait CatalystScan { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index bf6021e69238..e4250145a1ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -21,7 +21,7 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} import org.apache.spark.sql.execution.command.DDLUtils @@ -40,7 +40,7 @@ import org.apache.spark.util.Utils * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving final class DataStreamReader private[sql](sparkSession: SparkSession) extends Logging { /** * Specifies the input data source format. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index b36a8f3f6f15..5733258a6b31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -21,7 +21,7 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes @@ -39,7 +39,7 @@ import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { private val df = ds.toDF() @@ -365,7 +365,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.4.0 */ - @InterfaceStability.Evolving + @Evolving def foreachBatch(function: (Dataset[T], Long) => Unit): DataStreamWriter[T] = { this.source = "foreachBatch" if (function == null) throw new IllegalArgumentException("foreachBatch function cannot be null") @@ -386,7 +386,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.4.0 */ - @InterfaceStability.Evolving + @Evolving def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): DataStreamWriter[T] = { foreachBatch((batchDs: Dataset[T], batchId: Long) => function.call(batchDs, batchId)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala index e9510c903aca..ab68eba81b84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.streaming -import org.apache.spark.annotation.{Experimental, InterfaceStability} -import org.apache.spark.sql.KeyValueGroupedDataset +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState /** @@ -192,7 +191,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState * @since 2.2.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving trait GroupState[S] extends LogicalGroupState[S] { /** Whether state exists or not. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala index a033575d3d38..236bd55ee621 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala @@ -23,7 +23,7 @@ import scala.concurrent.duration.Duration import org.apache.commons.lang3.StringUtils -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.unsafe.types.CalendarInterval /** @@ -48,7 +48,7 @@ import org.apache.spark.unsafe.types.CalendarInterval * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving @deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") case class ProcessingTime(intervalMs: Long) extends Trigger { require(intervalMs >= 0, "the interval of trigger should not be negative") @@ -59,7 +59,7 @@ case class ProcessingTime(intervalMs: Long) extends Trigger { * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving @deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") object ProcessingTime { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index f2dfbe42260d..47ddc88e964e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import java.util.UUID -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.SparkSession /** @@ -27,7 +27,7 @@ import org.apache.spark.sql.SparkSession * All these methods are thread-safe. * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving trait StreamingQuery { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala index 03aeb14de502..646d6888b2a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving /** * Exception that stopped a [[StreamingQuery]]. Use `cause` get the actual exception @@ -28,7 +28,7 @@ import org.apache.spark.annotation.InterfaceStability * @param endOffset Ending offset in json of the range of data in exception occurred * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryException private[sql]( private val queryDebugString: String, val message: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 6aa82b89ede8..916d6a036596 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import java.util.UUID -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.scheduler.SparkListenerEvent /** @@ -28,7 +28,7 @@ import org.apache.spark.scheduler.SparkListenerEvent * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving abstract class StreamingQueryListener { import StreamingQueryListener._ @@ -67,14 +67,14 @@ abstract class StreamingQueryListener { * Companion object of [[StreamingQueryListener]] that defines the listener events. * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving object StreamingQueryListener { /** * Base type of [[StreamingQueryListener]] events * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving trait Event extends SparkListenerEvent /** @@ -84,7 +84,7 @@ object StreamingQueryListener { * @param name User-specified name of the query, null if not specified. * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving class QueryStartedEvent private[sql]( val id: UUID, val runId: UUID, @@ -95,7 +95,7 @@ object StreamingQueryListener { * @param progress The query progress updates. * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving class QueryProgressEvent private[sql](val progress: StreamingQueryProgress) extends Event /** @@ -107,7 +107,7 @@ object StreamingQueryListener { * with an exception. Otherwise, it will be `None`. * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving class QueryTerminatedEvent private[sql]( val id: UUID, val runId: UUID, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index cd52d991d55c..d9ea8dc9d4ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -25,7 +25,7 @@ import scala.collection.mutable import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker @@ -42,7 +42,7 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Logging { private[sql] val stateStoreCoordinator = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index a0c9bcc8929e..9dc62b7aac89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -22,7 +22,7 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving /** * Reports information about the instantaneous status of a streaming query. @@ -34,7 +34,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryStatus protected[sql]( val message: String, val isDataAvailable: Boolean, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index f2173aa1e59c..3cd6700efef5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -29,12 +29,12 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving /** * Information about updates made to stateful operators in a [[StreamingQuery]] during a trigger. */ -@InterfaceStability.Evolving +@Evolving class StateOperatorProgress private[sql]( val numRowsTotal: Long, val numRowsUpdated: Long, @@ -94,7 +94,7 @@ class StateOperatorProgress private[sql]( * @param sources detailed statistics on data being read from each of the streaming sources. * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryProgress private[sql]( val id: UUID, val runId: UUID, @@ -165,7 +165,7 @@ class StreamingQueryProgress private[sql]( * Spark. * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class SourceProgress protected[sql]( val description: String, val startOffset: String, @@ -209,7 +209,7 @@ class SourceProgress protected[sql]( * @param description Description of the source corresponding to this status. * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class SinkProgress protected[sql]( val description: String) extends Serializable { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index 1310fdfa1356..77ae047705de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.util import scala.collection.JavaConverters._ -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Evolving, Experimental} import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.SparkSession @@ -36,7 +36,7 @@ import org.apache.spark.util.{ListenerBus, Utils} * multiple different threads. */ @Experimental -@InterfaceStability.Evolving +@Evolving trait QueryExecutionListener { /** @@ -73,7 +73,7 @@ trait QueryExecutionListener { * Manager for [[QueryExecutionListener]]. See `org.apache.spark.sql.SQLContext.listenerManager`. */ @Experimental -@InterfaceStability.Evolving +@Evolving // The `session` is used to indicate which session carries this listener manager, and we only // catch SQL executions which are launched by the same session. // The `loadExtensions` flag is used to indicate whether we should load the pre-defined, diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java index bfe50c7810f7..fc2171dc99e4 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java @@ -148,7 +148,7 @@ public TColumn() { super(); } - public TColumn(_Fields setField, Object value) { + public TColumn(TColumn._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java index 44da2cdd089d..8504c6d608d4 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java @@ -142,7 +142,7 @@ public TColumnValue() { super(); } - public TColumnValue(_Fields setField, Object value) { + public TColumnValue(TColumnValue._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java index 4fe59b1c5146..fe2a211c4630 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java @@ -136,7 +136,7 @@ public TGetInfoValue() { super(); } - public TGetInfoValue(_Fields setField, Object value) { + public TGetInfoValue(TGetInfoValue._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java index af7c0b4f15d9..d0d70c127957 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java @@ -136,7 +136,7 @@ public TTypeEntry() { super(); } - public TTypeEntry(_Fields setField, Object value) { + public TTypeEntry(TTypeEntry._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java index 8c40687a0aab..a3e382937227 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java +++ b/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java @@ -112,7 +112,7 @@ public TTypeQualifierValue() { super(); } - public TTypeQualifierValue(_Fields setField, Object value) { + public TTypeQualifierValue(TTypeQualifierValue._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java index 9dd0efc03968..7e557aeccf5b 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java @@ -36,7 +36,7 @@ public abstract class AbstractService implements Service { /** * Service state: initially {@link STATE#NOTINITED}. */ - private STATE state = STATE.NOTINITED; + private Service.STATE state = STATE.NOTINITED; /** * Service name. @@ -70,7 +70,7 @@ public AbstractService(String name) { } @Override - public synchronized STATE getServiceState() { + public synchronized Service.STATE getServiceState() { return state; } @@ -159,7 +159,7 @@ public long getStartTime() { * if the service state is different from * the desired state */ - private void ensureCurrentState(STATE currentState) { + private void ensureCurrentState(Service.STATE currentState) { ServiceOperations.ensureCurrentState(state, currentState); } @@ -173,7 +173,7 @@ private void ensureCurrentState(STATE currentState) { * @param newState * new service state */ - private void changeState(STATE newState) { + private void changeState(Service.STATE newState) { state = newState; // notify listeners for (ServiceStateChangeListener l : listeners) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java index 5a508745414a..15551da4785f 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java @@ -71,7 +71,7 @@ public HiveConf getHiveConf() { } @Override - public STATE getServiceState() { + public Service.STATE getServiceState() { return service.getServiceState(); } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 2882672f327c..4f3914740ec2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener @@ -32,7 +32,7 @@ import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLo * Builder that produces a Hive-aware `SessionState`. */ @Experimental -@InterfaceStability.Unstable +@Unstable class HiveSessionStateBuilder(session: SparkSession, parentState: Option[SessionState] = None) extends BaseSessionStateBuilder(session, parentState) {