Repository: incubator-beam
Updated Branches:
  refs/heads/master f6ad58d60 -> 7e0cfe5b9


Added a where method on TypeDescriptor to allow substituting a type
parameter with a TypeDescriptor. In the process introduced a TypeParameter
class which represents the parameter type.
This is useful when having a type such as Set<T> to be able to specify
what T is as part of the TypeDescriptor information.


Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/699075e2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/699075e2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/699075e2

Branch: refs/heads/master
Commit: 699075e250973447a9cdce1eb9c52d3406788212
Parents: f6ad58d
Author: Jeremie Lenfant-Engelmann <jeremi...@google.com>
Authored: Thu Oct 20 19:29:39 2016 -0700
Committer: Kenneth Knowles <k...@google.com>
Committed: Tue Nov 1 15:14:24 2016 -0700

----------------------------------------------------------------------
 .../apache/beam/sdk/values/TypeDescriptor.java  | 35 +++++++++++----
 .../apache/beam/sdk/values/TypeDescriptors.java |  1 -
 .../apache/beam/sdk/values/TypeParameter.java   | 45 ++++++++++++++++++++
 .../beam/sdk/values/TypeDescriptorTest.java     | 12 ++++++
 4 files changed, 83 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/699075e2/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
index 724b8b6..6eabf42 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
@@ -20,7 +20,7 @@ package org.apache.beam.sdk.values;
 import com.google.common.collect.Lists;
 import com.google.common.reflect.Invokable;
 import com.google.common.reflect.Parameter;
-import com.google.common.reflect.TypeParameter;
+import com.google.common.reflect.TypeResolver;
 import com.google.common.reflect.TypeToken;
 import java.io.Serializable;
 import java.lang.reflect.Field;
@@ -171,14 +171,6 @@ public abstract class TypeDescriptor<T> implements 
Serializable {
   }
 
   /**
-   * Creates a new {@link SimpleTypeDescriptor} using the {@link #token}.
-   * Package visible so this isn't abused.
-   */
-  <X> TypeDescriptor<T> where(TypeParameter<X> typeParam, TypeDescriptor<X> 
typeDescriptor) {
-    return new SimpleTypeDescriptor<>(token.where(typeParam, 
typeDescriptor.token));
-  }
-
-  /**
    * Returns the {@link Type} represented by this {@link TypeDescriptor}.
    */
   public Type getType() {
@@ -322,6 +314,31 @@ public abstract class TypeDescriptor<T> implements 
Serializable {
     return classes;
   }
 
+  /**
+   * Returns a new {@code TypeDescriptor} where type variables represented by
+   * {@code typeParameter} are substituted by {@code typeDescriptor}. For 
example, it can be used to
+   * construct {@code Map<K, V>} for any {@code K} and {@code V} type: <pre> 
{@code
+   *   static <K, V> TypeDescriptor<Map<K, V>> mapOf(
+   *       TypeDescriptor<K> keyType, TypeDescriptor<V> valueType) {
+   *     return new TypeDescriptor<Map<K, V>>() {}
+   *         .where(new TypeParameter<K>() {}, keyType)
+   *         .where(new TypeParameter<V>() {}, valueType);
+   *   }}</pre>
+   *
+   * @param <X> The parameter type
+   * @param typeParameter the parameter type variable
+   * @param typeDescriptor the actual type to substitute
+   */
+  @SuppressWarnings("unchecked")
+  public <X> TypeDescriptor<T> where(TypeParameter<X> typeParameter,
+      TypeDescriptor<X> typeDescriptor) {
+    TypeResolver resolver =
+        new TypeResolver()
+            .where(
+                typeParameter.typeVariable, typeDescriptor.getType());
+    return (TypeDescriptor<T>) 
TypeDescriptor.of(resolver.resolveType(token.getType()));
+  }
+
   @Override
   public String toString() {
     return token.toString();

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/699075e2/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java
index 7a78131..b6b1a1a 100644
--- 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java
+++ 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.sdk.values;
 
-import com.google.common.reflect.TypeParameter;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/699075e2/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java 
b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java
new file mode 100644
index 0000000..a9a658e
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java
@@ -0,0 +1,45 @@
+package org.apache.beam.sdk.values;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.lang.reflect.TypeVariable;
+
+/**
+ * Captures a free type variable that can be used in {@link 
TypeDescriptor#where}. For example:
+ *
+ * <pre>   {@code
+ *   static <T> TypeDescriptor<List<T>> listOf(Class<T> elementType) {
+ *     return new TypeDescriptor<List<T>>() {}
+ *         .where(new TypeParameter<T>() {}, elementType);
+ *   }}</pre>
+ */
+public abstract class TypeParameter<T> {
+  final TypeVariable<?> typeVariable;
+
+  public TypeParameter() {
+    Type superclass = getClass().getGenericSuperclass();
+    checkArgument(superclass instanceof ParameterizedType, "%s isn't 
parameterized", superclass);
+    typeVariable = (TypeVariable<?>) ((ParameterizedType) 
superclass).getActualTypeArguments()[0];
+  }
+
+  @Override
+  public int hashCode() {
+    return typeVariable.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (!(obj instanceof TypeParameter)) {
+      return false;
+    }
+    TypeParameter<?> that = (TypeParameter<?>) obj;
+    return typeVariable.equals(that.typeVariable);
+  }
+
+  @Override
+  public String toString() {
+    return typeVariable.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/699075e2/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
----------------------------------------------------------------------
diff --git 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
index 39472f9..a9b44b7 100644
--- 
a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
+++ 
b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
@@ -194,4 +194,16 @@ public class TypeDescriptorTest {
         new TypeToken<List<Set<String>>>() {}.getType(),
         rememberer.descriptorByInstance.getType());
   }
+
+  @Test
+  public void testWhere() throws Exception {
+    useWhereMethodToDefineTypeParam(new TypeDescriptor<String>() {});
+  }
+
+  private <T> void useWhereMethodToDefineTypeParam(TypeDescriptor<T> 
parameterType) {
+    TypeDescriptor<Set<T>> typeDescriptor = new TypeDescriptor<Set<T>>() 
{}.where(
+        new TypeParameter<T>() {}, parameterType);
+    assertEquals(new TypeToken<Set<String>>() {}.getType(),
+        typeDescriptor.getType());
+  }
 }

Reply via email to