Skip to content

Commit

Permalink
[FLINK-8837][annotations] Add @experimental annotation and annotate s…
Browse files Browse the repository at this point in the history
…ome classes

This closes apache#5800.
  • Loading branch information
bowenli86 authored and zentol committed Apr 6, 2018
1 parent 7d6afed commit cdd2022
Show file tree
Hide file tree
Showing 5 changed files with 48 additions and 11 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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.flink.annotation;

import java.lang.annotation.Documented;
import java.lang.annotation.ElementType;
import java.lang.annotation.Target;

/**
* Annotation to mark classes for experimental use.
*
* <p>Classes with this annotation are neither battle-tested nor stable, and may be changed or removed
* in future versions.
*
* <p>This annotation also excludes classes with evolving interfaces / signatures
* annotated with {@link Public} and {@link PublicEvolving}.
*
*/
@Documented
@Target(ElementType.TYPE)
@Public
public @interface Experimental {
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.flink.streaming.api.datastream;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
Expand All @@ -39,11 +39,8 @@

/**
* A collection of utilities for {@link DataStream DataStreams}.
*
* <p>This experimental class is relocated from flink-streaming-contrib. Please see package-info.java
* for more information.
*/
@PublicEvolving
@Experimental
public final class DataStreamUtils {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.flink.streaming.experimental;

import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
Expand All @@ -34,7 +34,7 @@
* <p>This experimental class is relocated from flink-streaming-contrib. Please see package-info.java
* for more information.
*/
@Internal
@Experimental
public class CollectSink<IN> extends RichSinkFunction<IN> {

private static final long serialVersionUID = 1L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.flink.streaming.experimental;

import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputViewStreamWrapper;

Expand All @@ -41,7 +41,7 @@
*
* @param <T> The type of elements returned from the iterator.
*/
@PublicEvolving
@Experimental
public class SocketStreamIterator<T> implements Iterator<T> {

/** Server socket to listen at. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.flink.streaming.api.scala

import org.apache.flink.annotation.PublicEvolving
import org.apache.flink.annotation.Experimental
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.streaming.api.datastream.{DataStreamUtils => JavaStreamUtils}

Expand All @@ -33,7 +33,7 @@ import scala.reflect.ClassTag
*
* @param self DataStream
*/
@PublicEvolving
@Experimental
class DataStreamUtils[T: TypeInformation : ClassTag](val self: DataStream[T]) {

/**
Expand Down

0 comments on commit cdd2022

Please sign in to comment.