Skip to content

Commit

Permalink
[FLINK-3093] Introduce annotations for interface stability in remaini…
Browse files Browse the repository at this point in the history
…ng modules

This closes apache#1428
  • Loading branch information
rmetzger committed Feb 5, 2016
1 parent 2eb2a0e commit b54499b
Show file tree
Hide file tree
Showing 229 changed files with 1,077 additions and 92 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
* An experimental API might change between minor releases.
*/
@Documented
@Target({ ElementType.TYPE, ElementType.METHOD })
@Target({ ElementType.TYPE, ElementType.METHOD, ElementType.FIELD, ElementType.CONSTRUCTOR })
@Public
public @interface Experimental {
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
* Developer APIs are stable but internal to Flink and might change across releases.
*/
@Documented
@Target({ ElementType.TYPE, ElementType.METHOD })
@Target({ ElementType.TYPE, ElementType.METHOD, ElementType.CONSTRUCTOR })
@Public
public @interface Internal {
}
Original file line number Diff line number Diff line change
Expand Up @@ -32,4 +32,5 @@
*/
@Documented
@Target(ElementType.TYPE)
@Public
public @interface Public {}
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,6 @@
import org.apache.flink.core.fs.Path;
import org.apache.flink.util.InstantiationUtil;


public class AvroInputFormat<E> extends FileInputFormat<E> implements ResultTypeQueryable<E> {

private static final long serialVersionUID = 1L;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/


package org.apache.flink.api.java.io;


import org.apache.avro.Schema;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.io.DatumWriter;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.io.ObjectOutputStream;
import java.io.Serializable;

import org.apache.flink.annotation.Public;
import org.apache.flink.api.common.functions.RichFlatMapFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.tuple.Tuple2;
Expand All @@ -42,6 +43,7 @@
* This wrapper maps a Hadoop Mapper (mapred API) to a Flink FlatMapFunction.
*/
@SuppressWarnings("rawtypes")
@Public
public final class HadoopMapFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT>
extends RichFlatMapFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYOUT,VALUEOUT>>
implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.io.ObjectOutputStream;
import java.io.Serializable;

import org.apache.flink.annotation.Public;
import org.apache.flink.api.common.functions.GroupCombineFunction;
import org.apache.flink.api.common.functions.RichGroupReduceFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
Expand All @@ -45,6 +46,7 @@
* This wrapper maps a Hadoop Reducer and Combiner (mapred API) to a combinable Flink GroupReduceFunction.
*/
@SuppressWarnings("rawtypes")
@Public
public final class HadoopReduceCombineFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT>
extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>>
implements GroupCombineFunction<Tuple2<KEYIN,VALUEIN>, Tuple2<KEYIN,VALUEIN>>,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.io.ObjectOutputStream;
import java.io.Serializable;

import org.apache.flink.annotation.Public;
import org.apache.flink.api.common.functions.RichGroupReduceFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
Expand All @@ -44,6 +45,7 @@
* This wrapper maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction.
*/
@SuppressWarnings("rawtypes")
@Public
public final class HadoopReduceFunction<KEYIN, VALUEIN, KEYOUT, VALUEOUT>
extends RichGroupReduceFunction<Tuple2<KEYIN,VALUEIN>,Tuple2<KEYOUT,VALUEOUT>>
implements ResultTypeQueryable<Tuple2<KEYOUT,VALUEOUT>>, Serializable {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/


package org.apache.flink.addons.hbase;

import java.io.IOException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,7 @@ public interface RuntimeContext {
* @throws UnsupportedOperationException Thrown, if no partitioned state is available for the
* function (function is not part of a KeyedStream).
*/
@Experimental
<T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties);

/**
Expand Down Expand Up @@ -282,6 +283,7 @@ public interface RuntimeContext {
* @throws UnsupportedOperationException Thrown, if no partitioned state is available for the
* function (function is not part os a KeyedStream).
*/
@Experimental
<T> ListState<T> getListState(ListStateDescriptor<T> stateProperties);

/**
Expand Down Expand Up @@ -321,6 +323,7 @@ public interface RuntimeContext {
* @throws UnsupportedOperationException Thrown, if no partitioned state is available for the
* function (function is not part of a KeyedStream).
*/
@Experimental
<T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties);

/**
Expand Down Expand Up @@ -380,6 +383,7 @@ public interface RuntimeContext {
* @deprecated Use the more expressive {@link #getState(ValueStateDescriptor)} instead.
*/
@Deprecated
@Experimental
<S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState);

/**
Expand Down Expand Up @@ -432,5 +436,6 @@ public interface RuntimeContext {
* @deprecated Use the more expressive {@link #getState(ValueStateDescriptor)} instead.
*/
@Deprecated
@Experimental
<S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState);
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.concurrent.Future;

import com.google.common.base.Preconditions;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.TaskInfo;
import org.apache.flink.api.common.accumulators.Accumulator;
Expand Down Expand Up @@ -175,32 +176,37 @@ private <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name
}

@Override
@Experimental
public <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties) {
throw new UnsupportedOperationException(
"This state is only accessible by functions executed on a KeyedStream");
}

@Override
@Experimental
public <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties) {
throw new UnsupportedOperationException(
"This state is only accessible by functions executed on a KeyedStream");
}

@Override
@Experimental
public <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties) {
throw new UnsupportedOperationException(
"This state is only accessible by functions executed on a KeyedStream");
}

@Override
@Deprecated
@Experimental
public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
throw new UnsupportedOperationException(
"This state is only accessible by functions executed on a KeyedStream");
}

@Override
@Deprecated
@Experimental
public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
throw new UnsupportedOperationException(
"This state is only accessible by functions executed on a KeyedStream");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints;

import org.apache.flink.annotation.Public;
import org.apache.flink.core.fs.FileInputSplit;
import org.apache.flink.core.fs.Path;
import org.apache.flink.types.parser.FieldParser;
Expand All @@ -41,7 +40,6 @@
import java.util.Map;
import java.util.TreeMap;

@Public
public abstract class GenericCsvInputFormat<OT> extends DelimitedInputFormat<OT> {

private static final Logger LOG = LoggerFactory.getLogger(GenericCsvInputFormat.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.apache.flink.api.common.io.statistics;

import org.apache.flink.annotation.Experimental;
import org.apache.flink.annotation.Public;

/**
Expand All @@ -30,16 +31,19 @@ public interface BaseStatistics {
/**
* Constant indicating that the input size is unknown.
*/
@Experimental
public static final long SIZE_UNKNOWN = -1;

/**
* Constant indicating that the number of records is unknown;
*/
@Experimental
public static final long NUM_RECORDS_UNKNOWN = -1;

/**
* Constant indicating that average record width is unknown.
*/
@Experimental
public static final float AVG_RECORD_BYTES_UNKNOWN = -1.0f;

// --------------------------------------------------------------------------------------------
Expand All @@ -49,19 +53,22 @@ public interface BaseStatistics {
*
* @return The total size of the input, in bytes.
*/
@Experimental
public long getTotalInputSize();

/**
* Gets the number of records in the input (= base cardinality).
*
* @return The number of records in the input.
*/
@Experimental
public long getNumberOfRecords();

/**
* Gets the average width of a record, in bytes.
*
* @return The average width of a record in bytes.
*/
@Experimental
public float getAverageRecordWidth();
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,11 @@
import java.util.HashMap;
import java.util.Map;

import org.apache.flink.annotation.Public;
import org.apache.flink.api.common.operators.util.FieldSet;

/**
* Container for the semantic properties associated to a single input operator.
*/
@Public
public class SingleInputSemanticProperties implements SemanticProperties {
private static final long serialVersionUID = 1L;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.api.common.typeinfo;

import org.apache.flink.annotation.Public;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeutils.TypeComparator;

Expand All @@ -29,6 +30,7 @@
* In contrast to atomic types are composite types, where the type information is aware of the individual
* fields and individual fields may be used as a key.
*/
@Public
public interface AtomicType<T> {

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,15 @@
import java.util.Objects;

import com.google.common.base.Preconditions;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.annotation.Public;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.common.typeutils.base.array.StringArraySerializer;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeutils.base.GenericArraySerializer;

@Public
public final class BasicArrayTypeInfo<T, C> extends TypeInformation<T> {

private static final long serialVersionUID = 1L;
Expand Down Expand Up @@ -58,45 +61,54 @@ private BasicArrayTypeInfo(Class<T> arrayClass, BasicTypeInfo<C> componentInfo)
// --------------------------------------------------------------------------------------------

@Override
@Experimental
public boolean isBasicType() {
return false;
}

@Override
@Experimental
public boolean isTupleType() {
return false;
}

@Override
@Experimental
public int getArity() {
return 1;
}

@Override
@Experimental
public int getTotalFields() {
return 1;
}

@Override
@Experimental
public Class<T> getTypeClass() {
return this.arrayClass;
}

@Experimental
public Class<C> getComponentTypeClass() {
return this.componentInfo.getTypeClass();
}


@Experimental
public TypeInformation<C> getComponentInfo() {
return componentInfo;
}

@Override
@Experimental
public boolean isKeyType() {
return false;
}

@Override
@SuppressWarnings("unchecked")
@Experimental
public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
// special case the string array
if (componentInfo.getTypeClass().equals(String.class)) {
Expand Down Expand Up @@ -140,6 +152,7 @@ public String toString() {
// --------------------------------------------------------------------------------------------

@SuppressWarnings("unchecked")
@Experimental
public static <X, C> BasicArrayTypeInfo<X, C> getInfoFor(Class<X> type) {
if (!type.isArray()) {
throw new InvalidTypesException("The given class is no array.");
Expand Down
Loading

0 comments on commit b54499b

Please sign in to comment.