This is an automated email from the ASF dual-hosted git repository.
boyuanz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push:
new c3367f7 Remove Experimental annotation from SDF related APIs.
new 2462fe9 Merge pull request #13199 from [BEAM-11132] Remove Experimental annotation
from SDF related APIs.
c3367f7 is described below
commit c3367f7db1c6e8ca2bc26f827dbb8a2c2f32c002
Author: Boyuan Zhang <boyuanz@google.com>
AuthorDate: Mon Oct 26 12:30:47 2020 -0700
Remove Experimental annotation from SDF related APIs.
---
.../java/org/apache/beam/sdk/annotations/Experimental.java | 7 -------
.../core/src/main/java/org/apache/beam/sdk/io/AvroIO.java | 7 ++-----
.../core/src/main/java/org/apache/beam/sdk/io/FileIO.java | 5 ++---
.../core/src/main/java/org/apache/beam/sdk/io/TextIO.java | 5 ++---
.../java/org/apache/beam/sdk/transforms/Deduplicate.java | 3 ---
.../src/main/java/org/apache/beam/sdk/transforms/DoFn.java | 13 -------------
.../org/apache/beam/sdk/transforms/PeriodicSequence.java | 3 ---
.../src/main/java/org/apache/beam/sdk/transforms/Watch.java | 3 ---
.../sdk/transforms/splittabledofn/ByteKeyRangeTracker.java | 3 ---
.../splittabledofn/GrowableOffsetRangeTracker.java | 3 ---
.../sdk/transforms/splittabledofn/HasDefaultTracker.java | 4 ----
.../splittabledofn/HasDefaultWatermarkEstimator.java | 3 ---
.../transforms/splittabledofn/ManualWatermarkEstimator.java | 3 ---
.../sdk/transforms/splittabledofn/OffsetRangeTracker.java | 3 ---
.../sdk/transforms/splittabledofn/RestrictionTracker.java | 3 ---
.../beam/sdk/transforms/splittabledofn/SplitResult.java | 3 ---
.../TimestampObservingWatermarkEstimator.java | 3 ---
.../sdk/transforms/splittabledofn/WatermarkEstimator.java | 3 ---
.../sdk/transforms/splittabledofn/WatermarkEstimators.java | 3 ---
.../beam/sdk/transforms/splittabledofn/package-info.java | 3 ---
.../src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java | 1 -
sdks/python/apache_beam/io/iobase.py | 7 +------
22 files changed, 7 insertions(+), 84 deletions(-)
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java
index b9a5a49..48ff40d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/annotations/Experimental.java
@@ -74,13 +74,6 @@ public @interface Experimental {
/** Experimental APIs related to customizing the output time for computed values. */
OUTPUT_TIME,
- /**
- * <a href="https://s.apache.org/splittable-do-fn">Splittable DoFn</a>. See
<a
- * href="https://beam.apache.org/documentation/runners/capability-matrix/">capability
matrix</a>
- * for runner support.
- */
- SPLITTABLE_DO_FN,
-
/** Metrics-related experimental APIs. */
METRICS,
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index 97fd464..8fb4186 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -652,9 +652,9 @@ public class AvroIO {
* interval, until the given termination condition is reached. The returned {@link PCollection}
* is unbounded.
*
- * <p>This works only in runners supporting {@link Kind#SPLITTABLE_DO_FN}.
+ * <p>This works only in runners supporting splittable {@link
+ * org.apache.beam.sdk.transforms.DoFn}.
*/
- @Experimental(Kind.SPLITTABLE_DO_FN)
public Read<T> watchForNewFiles(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withMatchConfiguration(
@@ -877,7 +877,6 @@ public class AvroIO {
}
/** Like {@link Read#watchForNewFiles}. */
- @Experimental(Kind.SPLITTABLE_DO_FN)
public ReadAll<T> watchForNewFiles(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withMatchConfiguration(
@@ -1009,7 +1008,6 @@ public class AvroIO {
}
/** Like {@link Read#watchForNewFiles}. */
- @Experimental(Kind.SPLITTABLE_DO_FN)
public Parse<T> watchForNewFiles(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withMatchConfiguration(
@@ -1189,7 +1187,6 @@ public class AvroIO {
}
/** Like {@link Read#watchForNewFiles}. */
- @Experimental(Kind.SPLITTABLE_DO_FN)
public ParseAll<T> watchForNewFiles(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withMatchConfiguration(
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java
index 07e6b10..8a1e129 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java
@@ -562,9 +562,9 @@ public class FileIO {
/**
* See {@link MatchConfiguration#continuously}. The returned {@link PCollection} is unbounded.
*
- * <p>This works only in runners supporting {@link Experimental.Kind#SPLITTABLE_DO_FN}.
+ * <p>This works only in runners supporting splittable {@link
+ * org.apache.beam.sdk.transforms.DoFn}.
*/
- @Experimental(Kind.SPLITTABLE_DO_FN)
public Match continuously(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withConfiguration(getConfiguration().continuously(pollInterval, terminationCondition));
@@ -613,7 +613,6 @@ public class FileIO {
}
/** Like {@link Match#continuously}. */
- @Experimental(Kind.SPLITTABLE_DO_FN)
public MatchAll continuously(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withConfiguration(getConfiguration().continuously(pollInterval, terminationCondition));
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index 4d04b1e..baae960 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -352,9 +352,9 @@ public class TextIO {
/**
* See {@link MatchConfiguration#continuously}.
*
- * <p>This works only in runners supporting {@link Kind#SPLITTABLE_DO_FN}.
+ * <p>This works only in runners supporting splittable {@link
+ * org.apache.beam.sdk.transforms.DoFn}.
*/
- @Experimental(Kind.SPLITTABLE_DO_FN)
public Read watchForNewFiles(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withMatchConfiguration(
@@ -497,7 +497,6 @@ public class TextIO {
}
/** Same as {@link Read#watchForNewFiles(Duration, TerminationCondition)}. */
- @Experimental(Kind.SPLITTABLE_DO_FN)
public ReadAll watchForNewFiles(
Duration pollInterval, TerminationCondition<String, ?> terminationCondition)
{
return withMatchConfiguration(
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Deduplicate.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Deduplicate.java
index 8f881cc..42b9e96 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Deduplicate.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Deduplicate.java
@@ -17,8 +17,6 @@
*/
package org.apache.beam.sdk.transforms;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.coders.BooleanCoder;
import org.apache.beam.sdk.coders.Coder;
import org.apache.beam.sdk.coders.KvCoder;
@@ -66,7 +64,6 @@ import org.joda.time.Duration;
* words.apply(Deduplicate.<String>values());
* }</pre>
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
public final class Deduplicate {
/** The default is the {@link TimeDomain#PROCESSING_TIME processing time domain}. */
public static final TimeDomain DEFAULT_TIME_DOMAIN = TimeDomain.PROCESSING_TIME;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 115b480..7e27b5f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -938,7 +938,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface GetInitialRestriction {}
/**
@@ -994,7 +993,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface GetSize {}
/**
@@ -1011,7 +1009,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface GetRestrictionCoder {}
/**
@@ -1057,7 +1054,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface SplitRestriction {}
/**
@@ -1110,7 +1106,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface TruncateRestriction {}
/**
@@ -1148,7 +1143,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface NewTracker {}
/**
@@ -1189,7 +1183,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface GetInitialWatermarkEstimatorState {}
/**
@@ -1206,7 +1199,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface GetWatermarkEstimatorStateCoder {}
/**
@@ -1251,7 +1243,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.METHOD)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface NewWatermarkEstimator {}
/**
@@ -1262,7 +1253,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.PARAMETER)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface WatermarkEstimatorState {}
/**
@@ -1274,7 +1264,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface BoundedPerElement {}
/**
@@ -1286,7 +1275,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
@Documented
@Retention(RetentionPolicy.RUNTIME)
@Target(ElementType.TYPE)
- @Experimental(Kind.SPLITTABLE_DO_FN)
public @interface UnboundedPerElement {}
// This can't be put into ProcessContinuation itself due to the following problem:
@@ -1301,7 +1289,6 @@ public abstract class DoFn<InputT extends @Nullable Object, OutputT
extends @Nul
* <p>If the {@link ProcessElement} call completes because of a failed {@code tryClaim()}
call on
* the {@link RestrictionTracker}, then the call MUST return {@link #stop()}.
*/
- @Experimental(Kind.SPLITTABLE_DO_FN)
@AutoValue
public abstract static class ProcessContinuation {
/** Indicates that there is no more work to be done for the current element. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
index c15d968..f2d5e35 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
@@ -22,7 +22,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Prec
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
import java.util.Objects;
-import org.apache.beam.sdk.annotations.Experimental;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.sdk.schemas.JavaFieldSchema;
import org.apache.beam.sdk.schemas.annotations.DefaultSchema;
@@ -42,7 +41,6 @@ import org.joda.time.Instant;
* reach given timestamp. Transform will not output elements prior to target time. Transform
can
* output elements at any time after target time.
*/
-@Experimental(Experimental.Kind.SPLITTABLE_DO_FN)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
@@ -92,7 +90,6 @@ public class PeriodicSequence
return new PeriodicSequence();
}
- @Experimental(Experimental.Kind.SPLITTABLE_DO_FN)
public static class OutputRangeTracker extends RestrictionTracker<OffsetRange, Long>
implements RestrictionTracker.HasProgress {
private OffsetRange range;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java
index 86d6d5c..42d8b74 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java
@@ -36,8 +36,6 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.coders.AtomicCoder;
import org.apache.beam.sdk.coders.CannotProvideCoderException;
import org.apache.beam.sdk.coders.Coder;
@@ -125,7 +123,6 @@ import org.slf4j.LoggerFactory;
* <p>Note: This transform works only in runners supporting Splittable DoFn: see <a
* href="https://beam.apache.org/documentation/runners/capability-matrix/">capability
matrix</a>.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
@SuppressWarnings({
"nullness", // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
"rawtypes"
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java
index 4b76b47..c057764 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java
@@ -21,8 +21,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Prec
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.io.range.ByteKey;
import org.apache.beam.sdk.io.range.ByteKeyRange;
import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.HasProgress;
@@ -39,7 +37,6 @@ import org.checkerframework.checker.nullness.qual.Nullable;
* <p>Note, one can complete a range by claiming the {@link ByteKey#EMPTY} once one
runs out of keys
* to process.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
index 039f179..d7bcec1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
@@ -21,8 +21,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Prec
import java.math.BigDecimal;
import java.math.MathContext;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers;
@@ -35,7 +33,6 @@ import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers;
*
* <p>The growable range is marked as done by claiming {@code Long.MAX_VALUE}.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultTracker.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultTracker.java
index c9fc3fc..8badd5c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultTracker.java
@@ -17,15 +17,11 @@
*/
package org.apache.beam.sdk.transforms.splittabledofn;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
-
/**
* Interface for restrictions for which a default implementation of {@link
* org.apache.beam.sdk.transforms.DoFn.NewTracker} is available, depending only on the restriction
* itself.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
public interface HasDefaultTracker<
RestrictionT extends HasDefaultTracker<RestrictionT, TrackerT>,
TrackerT extends RestrictionTracker<RestrictionT, ?>> {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultWatermarkEstimator.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultWatermarkEstimator.java
index fac64ad..92a319d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultWatermarkEstimator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/HasDefaultWatermarkEstimator.java
@@ -17,15 +17,12 @@
*/
package org.apache.beam.sdk.transforms.splittabledofn;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.transforms.DoFn;
/**
* Interface for watermark estimator state for which a default implementation of {@link
* DoFn.NewWatermarkEstimator} is available, depending only on the watermark estimator state
itself.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
public interface HasDefaultWatermarkEstimator<
WatermarkEstimatorStateT,
WatermarkEstimatorT extends WatermarkEstimator<WatermarkEstimatorStateT>> {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ManualWatermarkEstimator.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ManualWatermarkEstimator.java
index 0121af0..1e6967c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ManualWatermarkEstimator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ManualWatermarkEstimator.java
@@ -17,8 +17,6 @@
*/
package org.apache.beam.sdk.transforms.splittabledofn;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.transforms.DoFn;
import org.joda.time.Instant;
@@ -27,7 +25,6 @@ import org.joda.time.Instant;
* DoFn} must invoke {@link #setWatermark} to advance the watermark. See {@link
* WatermarkEstimators.Manual} for a concrete implementation.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
public interface ManualWatermarkEstimator<WatermarkEstimatorStateT>
extends WatermarkEstimator<WatermarkEstimatorStateT> {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
index 015f06a..0c91a55 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
@@ -23,8 +23,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Prec
import java.math.BigDecimal;
import java.math.MathContext;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.io.range.OffsetRange;
import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.HasProgress;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
@@ -37,7 +35,6 @@ import org.checkerframework.checker.nullness.qual.Nullable;
* <p>The smallest offset is {@code Long.MIN_VALUE} and the largest offset is {@code
Long.MAX_VALUE
* - 1}.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
index 77a0592..669600d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
@@ -18,8 +18,6 @@
package org.apache.beam.sdk.transforms.splittabledofn;
import com.google.auto.value.AutoValue;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.transforms.DoFn;
import org.checkerframework.checker.nullness.qual.Nullable;
@@ -34,7 +32,6 @@ import org.checkerframework.checker.nullness.qual.Nullable;
* of workers and/or splitting may result if the progress is an inaccurate representation
of the
* known amount of completed and remaining work.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
@SuppressWarnings({
"rawtypes" // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
})
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResult.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResult.java
index 8ff8174..8c44ae4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResult.java
@@ -18,12 +18,9 @@
package org.apache.beam.sdk.transforms.splittabledofn;
import com.google.auto.value.AutoValue;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.checkerframework.checker.nullness.qual.Nullable;
/** A representation of a split result. */
-@Experimental(Kind.SPLITTABLE_DO_FN)
@AutoValue
@SuppressWarnings({
"rawtypes" // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/TimestampObservingWatermarkEstimator.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/TimestampObservingWatermarkEstimator.java
index b217690..736702c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/TimestampObservingWatermarkEstimator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/TimestampObservingWatermarkEstimator.java
@@ -17,8 +17,6 @@
*/
package org.apache.beam.sdk.transforms.splittabledofn;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.transforms.DoFn;
import org.joda.time.Instant;
@@ -26,7 +24,6 @@ import org.joda.time.Instant;
* A {@link WatermarkEstimator} that observes the timestamps of all records output from a
{@link
* DoFn}.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
public interface TimestampObservingWatermarkEstimator<WatermarkEstimatorStateT>
extends WatermarkEstimator<WatermarkEstimatorStateT> {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimator.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimator.java
index 92d9248..4bf9269 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimator.java
@@ -17,8 +17,6 @@
*/
package org.apache.beam.sdk.transforms.splittabledofn;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.transforms.DoFn;
import org.joda.time.Instant;
@@ -26,7 +24,6 @@ import org.joda.time.Instant;
* A {@link WatermarkEstimator} which is used for estimating output watermarks of a splittable
* {@link DoFn}. See {@link WatermarkEstimators} for commonly used watermark estimators.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
public interface WatermarkEstimator<WatermarkEstimatorStateT> {
/**
* Return estimated output watermark. This method must return monotonically increasing
watermarks
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java
index 75a115b..63c63b8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java
@@ -19,8 +19,6 @@ package org.apache.beam.sdk.transforms.splittabledofn;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.joda.time.Instant;
@@ -29,7 +27,6 @@ import org.joda.time.Instant;
* A set of {@link WatermarkEstimator}s that users can use to advance the output watermark
for their
* associated {@link DoFn splittable DoFn}s.
*/
-@Experimental(Kind.SPLITTABLE_DO_FN)
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
})
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/package-info.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/package-info.java
index 5400c49..e2722b4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/package-info.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/package-info.java
@@ -20,10 +20,7 @@
* {@link org.apache.beam.sdk.transforms.DoFn}.
*/
@DefaultAnnotation(NonNull.class)
-@Experimental(Kind.SPLITTABLE_DO_FN)
package org.apache.beam.sdk.transforms.splittabledofn;
import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
-import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.annotations.Experimental.Kind;
import org.checkerframework.checker.nullness.qual.NonNull;
diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
index 8cb509e..7d4edf8 100644
--- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
+++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
@@ -347,7 +347,6 @@ public class HBaseIO {
* A {@link PTransform} that works like {@link #read}, but executes read operations coming
from a
* {@link PCollection} of {@link Read}.
*/
- @Experimental(Kind.SPLITTABLE_DO_FN)
public static ReadAll readAll() {
return new ReadAll();
}
diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py
index 5501a8e..24bb91c 100644
--- a/sdks/python/apache_beam/io/iobase.py
+++ b/sdks/python/apache_beam/io/iobase.py
@@ -1206,8 +1206,6 @@ class RestrictionTracker(object):
See following documents for more details.
* https://s.apache.org/splittable-do-fn
* https://s.apache.org/splittable-do-fn-python-sdk
-
- Experimental; no backwards-compatibility guarantees.
"""
def current_restriction(self):
"""Returns the current restriction.
@@ -1373,10 +1371,7 @@ class WatermarkEstimator(object):
class RestrictionProgress(object):
- """Used to record the progress of a restriction.
-
- Experimental; no backwards-compatibility guarantees.
- """
+ """Used to record the progress of a restriction."""
def __init__(self, **kwargs):
# Only accept keyword arguments.
self._fraction = kwargs.pop('fraction', None)
|