diff --git a/CHANGES.md b/CHANGES.md index 5227bddba7426..bb5e8f64fc96b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -56,6 +56,7 @@ * New highly anticipated feature X added to Python SDK ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)). * New highly anticipated feature Y added to Java SDK ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)). +* The Beam Java API for Calcite SqlTransform is no longer experimental ([BEAM-12680](https://issues.apache.org/jira/browse/BEAM-12680)). ## I/Os diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index 397d1171b5423..5ddaca5baaf11 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -107,12 +107,7 @@ * } */ @AutoValue -@Experimental @AutoValue.CopyAnnotations -@SuppressWarnings({ - "rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556) - "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402) -}) public abstract class SqlTransform extends PTransform> { static final String PCOLLECTION_NAME = "PCOLLECTION"; @@ -122,9 +117,9 @@ public abstract class SqlTransform extends PTransform> abstract QueryParameters queryParameters(); - abstract List udfDefinitions(); + abstract @Experimental List udfDefinitions(); - abstract List udafDefinitions(); + abstract @Experimental List udafDefinitions(); abstract boolean autoLoading(); @@ -154,8 +149,9 @@ public PCollection expand(PInput input) { tableProviderMap().forEach(sqlEnvBuilder::addSchema); - if (defaultTableProvider() != null) { - sqlEnvBuilder.setCurrentSchema(defaultTableProvider()); + final @Nullable String defaultTableProvider = defaultTableProvider(); + if (defaultTableProvider != null) { + sqlEnvBuilder.setCurrentSchema(defaultTableProvider); } sqlEnvBuilder.setQueryPlannerClassName( @@ -239,6 +235,7 @@ public SqlTransform withDefaultTableProvider(String name, TableProvider tablePro return withTableProvider(name, tableProvider).toBuilder().setDefaultTableProvider(name).build(); } + @Experimental public SqlTransform withQueryPlannerClass(Class clazz) { return toBuilder().setQueryPlannerClassName(clazz.getName()).build(); } @@ -265,6 +262,7 @@ public SqlTransform withAutoLoading(boolean autoLoading) { * *

Refer to {@link BeamSqlUdf} for more about how to implement a UDF in BeamSql. */ + @Experimental public SqlTransform registerUdf(String functionName, Class clazz) { return registerUdf(functionName, clazz, BeamSqlUdf.UDF_METHOD); } @@ -273,6 +271,7 @@ public SqlTransform registerUdf(String functionName, Class * Register {@link SerializableFunction} as a UDF function used in this query. Note, {@link * SerializableFunction} must have a constructor without arguments. */ + @Experimental public SqlTransform registerUdf(String functionName, SerializableFunction sfn) { return registerUdf(functionName, sfn.getClass(), "apply"); } @@ -288,6 +287,7 @@ private SqlTransform registerUdf(String functionName, Class clazz, String met } /** register a {@link Combine.CombineFn} as UDAF function used in this query. */ + @Experimental public SqlTransform registerUdaf(String functionName, Combine.CombineFn combineFn) { ImmutableList newUdafs = ImmutableList.builder() @@ -311,6 +311,7 @@ static Builder builder() { } @AutoValue.Builder + @AutoValue.CopyAnnotations abstract static class Builder { abstract Builder setQueryString(String queryString); @@ -318,9 +319,9 @@ abstract static class Builder { abstract Builder setDdlStrings(List ddlStrings); - abstract Builder setUdfDefinitions(List udfDefinitions); + abstract @Experimental Builder setUdfDefinitions(List udfDefinitions); - abstract Builder setUdafDefinitions(List udafDefinitions); + abstract @Experimental Builder setUdafDefinitions(List udafDefinitions); abstract Builder setAutoLoading(boolean autoLoading); @@ -335,7 +336,7 @@ abstract static class Builder { @AutoValue @AutoValue.CopyAnnotations - @SuppressWarnings({"rawtypes"}) + @Experimental abstract static class UdfDefinition { abstract String udfName(); @@ -350,7 +351,7 @@ static UdfDefinition of(String udfName, Class clazz, String methodName) { @AutoValue @AutoValue.CopyAnnotations - @SuppressWarnings({"rawtypes"}) + @Experimental abstract static class UdafDefinition { abstract String udafName(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java index 8b3d95d26b36a..85c917d302323 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java @@ -19,6 +19,7 @@ import java.util.Collections; import java.util.Map; +import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.extensions.sql.BeamSqlUdf; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -27,6 +28,7 @@ @SuppressWarnings({ "rawtypes" // TODO(https://issues.apache.org/jira/browse/BEAM-10556) }) +@Experimental public interface UdfUdafProvider { /** For UDFs implement {@link BeamSqlUdf}. */ default Map> getBeamSqlUdfs() {