[
https://issues.apache.org/jira/browse/BEAM-4769?focusedWorklogId=125626&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-125626
]
ASF GitHub Bot logged work on BEAM-4769:
----------------------------------------
Author: ASF GitHub Bot
Created on: 20/Jul/18 18:51
Start Date: 20/Jul/18 18:51
Worklog Time Spent: 10m
Work Description: lukecwik closed pull request #5967: [BEAM-4769] Fix
issue when performing coder inference for wildcard types.
URL: https://github.com/apache/beam/pull/5967
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
index 167ff84f01c..865b2488608 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
@@ -621,11 +621,7 @@ private static boolean isNullOrEmpty(Collection<?> c) {
} else if (type instanceof WildcardType) {
// No coder for an unknown generic type.
throw new CannotProvideCoderException(
- String.format(
- "Cannot provide a coder for type variable %s"
- + " (declared by %s) because the actual type is unknown due
to erasure.",
- type, ((TypeVariable<?>) type).getGenericDeclaration()),
- ReasonCode.TYPE_ERASURE);
+ String.format("Cannot provide a coder for wildcard type %s.", type),
ReasonCode.UNKNOWN);
} else {
throw new RuntimeException("Internal error: unexpected kind of Type: " +
type);
}
diff --git
a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
index de561dd1779..4b21fd29fbf 100644
---
a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
+++
b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
@@ -27,6 +27,7 @@
import java.io.InputStream;
import java.io.OutputStream;
import java.io.Serializable;
+import java.lang.reflect.ParameterizedType;
import java.lang.reflect.Type;
import java.util.Collections;
import java.util.List;
@@ -149,6 +150,21 @@ public void testParameterizedDefaultCoderUnknown() throws
Exception {
registry.getCoder(listUnknownToken);
}
+ @Test
+ public void testParameterizedWildcardTypeIsUnknown() throws Exception {
+ CoderRegistry registry = CoderRegistry.createDefault();
+ TypeDescriptor<List<? extends MyValue>> wildcardUnknownToken =
+ new TypeDescriptor<List<? extends MyValue>>() {};
+
+ thrown.expect(CannotProvideCoderException.class);
+ thrown.expectMessage(
+ String.format(
+ "Cannot provide coder for parameterized type %s: Cannot provide a
coder for wildcard type %s.",
+ wildcardUnknownToken,
+ ((ParameterizedType)
wildcardUnknownToken.getType()).getActualTypeArguments()[0]));
+ registry.getCoder(wildcardUnknownToken);
+ }
+
@Test
public void testTypeParameterInferenceForward() throws Exception {
CoderRegistry registry = CoderRegistry.createDefault();
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 125626)
Time Spent: 0.5h (was: 20m)
> Types$WildcardTypeImpl cannot be cast to java.lang.reflect.TypeVariable
> -----------------------------------------------------------------------
>
> Key: BEAM-4769
> URL: https://issues.apache.org/jira/browse/BEAM-4769
> Project: Beam
> Issue Type: Bug
> Components: sdk-java-core
> Affects Versions: 2.5.0
> Reporter: Dzmitry Lazerka
> Assignee: Luke Cwik
> Priority: Minor
> Time Spent: 0.5h
> Remaining Estimate: 0h
>
> Hi,
> I'm getting the following exception, probably when Beam cannot find a coder
> for a wildcard generic type (although I don't think I use any, gotta find).
> {code:java}
> Exception in thread "main" java.lang.ClassCastException:
> org.apache.beam.repackaged.beam_sdks_java_core.com.google.common.reflect.Types$WildcardTypeImpl
> cannot be cast to java.lang.reflect.TypeVariable
> at
> org.apache.beam.sdk.coders.CoderRegistry.getCoderFromTypeDescriptor(CoderRegistry.java:625)
> at
> org.apache.beam.sdk.coders.CoderRegistry.getCoderFromParameterizedType(CoderRegistry.java:658)
> at
> org.apache.beam.sdk.coders.CoderRegistry.getCoderFromTypeDescriptor(CoderRegistry.java:620)
> at
> org.apache.beam.sdk.coders.CoderRegistry.getCoderFromParameterizedType(CoderRegistry.java:658)
> at
> org.apache.beam.sdk.coders.CoderRegistry.getCoderFromTypeDescriptor(CoderRegistry.java:620)
> at org.apache.beam.sdk.coders.CoderRegistry.getCoder(CoderRegistry.java:254)
> at
> org.apache.beam.sdk.values.PCollection.inferCoderOrFail(PCollection.java:149)
> at
> org.apache.beam.sdk.values.PCollection.finishSpecifyingOutput(PCollection.java:89)
> at
> org.apache.beam.sdk.runners.TransformHierarchy.setOutput(TransformHierarchy.java:208)
> at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:538)
> at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:472)
> at org.apache.beam.sdk.values.PCollection.apply(PCollection.java:286)
> at org.apache.beam.sdk.transforms.ParDo$SingleOutput.expand(ParDo.java:649)
> at org.apache.beam.sdk.transforms.ParDo$SingleOutput.expand(ParDo.java:584)
> at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
> at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:491)
> at org.apache.beam.sdk.values.PCollection.apply(PCollection.java:299)
> at org.apache.beam.sdk.transforms.MapElements.expand(MapElements.java:123)
> at org.apache.beam.sdk.transforms.MapElements.expand(MapElements.java:36)
> at org.apache.beam.sdk.Pipeline.applyInternal(Pipeline.java:537)
> at org.apache.beam.sdk.Pipeline.applyTransform(Pipeline.java:491)
> at org.apache.beam.sdk.values.PCollection.apply(PCollection.java:299)
> {code}
> The error seems to be in the following lines of CoderRegistry.java @ 629:
> {code:java|title=CoderRegistry.java|borderStyle=solid}
> } else if (type instanceof TypeVariable) {
> coder = getCoderFromFactories(typeDescriptor, Collections.emptyList());
> } else if (type instanceof WildcardType) {
> // No coder for an unknown generic type.
> throw new CannotProvideCoderException(
> String.format("Cannot provide a coder for type variable %s"
> + " (declared by %s) because the actual type is unknown due to
> erasure.",
> type,
> ((TypeVariable<?>) type).getGenericDeclaration()),
> ReasonCode.TYPE_ERASURE);
> {code}
> It tries to cast a WildcardType to TypeVariable<?>, although that should've
> been caught above on line 621.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)