This is an automated email from the ASF dual-hosted git repository.
ibzib 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 be92829 [BEAM-12609] Make type parameter part of PushdownProjector
interface.
new 9cdf661 Merge pull request #15297 from ibzib/BEAM-12609-t
be92829 is described below
commit be928296dfa8a4268e76cf4bb20ff0406aa62fbd
Author: Kyle Weaver <[email protected]>
AuthorDate: Fri Aug 6 15:22:32 2021 -0700
[BEAM-12609] Make type parameter part of PushdownProjector interface.
---
.../java/org/apache/beam/sdk/schemas/io/PushdownProjector.java | 4 ++--
.../sql/meta/provider/SchemaIOTableProviderWrapper.java | 10 ++++------
.../sql/meta/provider/TestSchemaIOTableProviderWrapper.java | 7 ++-----
3 files changed, 8 insertions(+), 13 deletions(-)
diff --git
a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/PushdownProjector.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/PushdownProjector.java
index 0ae7808..5bfb55e 100644
---
a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/PushdownProjector.java
+++
b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/io/PushdownProjector.java
@@ -32,12 +32,12 @@ import org.apache.beam.sdk.values.Row;
* return a {@link PushdownProjector} to which a projection may be applied
later.
*/
@Experimental
-public interface PushdownProjector {
+public interface PushdownProjector<InputT extends PInput> {
/**
* Returns a {@link PTransform} that will execute the projection specified
by the {@link
* FieldAccessDescriptor}.
*/
- PTransform<? extends PInput, PCollection<Row>> withProjectionPushdown(
+ PTransform<InputT, PCollection<Row>> withProjectionPushdown(
FieldAccessDescriptor fieldAccessDescriptor);
/**
diff --git
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java
index 34b640e..88f070c 100644
---
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java
+++
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java
@@ -136,14 +136,12 @@ public abstract class SchemaIOTableProviderWrapper
extends InMemoryMetaTableProv
}
if (!fieldNames.isEmpty()) {
if (readerTransform instanceof PushdownProjector) {
- PushdownProjector pushdownProjector = (PushdownProjector)
readerTransform;
- FieldAccessDescriptor fieldAccessDescriptor =
- FieldAccessDescriptor.withFieldNames(fieldNames);
// The pushdown must return a PTransform that can be applied to a
PBegin, or this cast
// will fail.
- readerTransform =
- (PTransform<PBegin, PCollection<Row>>)
-
pushdownProjector.withProjectionPushdown(fieldAccessDescriptor);
+ PushdownProjector<PBegin> pushdownProjector =
(PushdownProjector<PBegin>) readerTransform;
+ FieldAccessDescriptor fieldAccessDescriptor =
+ FieldAccessDescriptor.withFieldNames(fieldNames);
+ readerTransform =
pushdownProjector.withProjectionPushdown(fieldAccessDescriptor);
} else {
throw new UnsupportedOperationException(
String.format("%s does not support projection pushdown.",
this.getClass()));
diff --git
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/TestSchemaIOTableProviderWrapper.java
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/TestSchemaIOTableProviderWrapper.java
index 835fc5b..a198460 100644
---
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/TestSchemaIOTableProviderWrapper.java
+++
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/TestSchemaIOTableProviderWrapper.java
@@ -30,17 +30,14 @@ import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.Row;
-import
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
import org.checkerframework.checker.nullness.qual.Nullable;
/**
* A mock {@link
org.apache.beam.sdk.extensions.sql.meta.provider.SchemaIOTableProviderWrapper}
that
* reads in-memory data for testing.
*/
-@VisibleForTesting
public class TestSchemaIOTableProviderWrapper extends
SchemaIOTableProviderWrapper {
private static final List<Row> rows = new ArrayList<>();
@@ -109,7 +106,7 @@ public class TestSchemaIOTableProviderWrapper extends
SchemaIOTableProviderWrapp
* {@link Select}.
*/
private class TestPushdownProjector extends PTransform<PBegin,
PCollection<Row>>
- implements PushdownProjector {
+ implements PushdownProjector<PBegin> {
/** The schema of the input data. */
private final Schema schema;
/** The fields to be projected. */
@@ -121,7 +118,7 @@ public class TestSchemaIOTableProviderWrapper extends
SchemaIOTableProviderWrapp
}
@Override
- public PTransform<? extends PInput, PCollection<Row>>
withProjectionPushdown(
+ public PTransform<PBegin, PCollection<Row>> withProjectionPushdown(
FieldAccessDescriptor fieldAccessDescriptor) {
return new TestPushdownProjector(schema, fieldAccessDescriptor);
}