You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2019/04/02 19:12:29 UTC

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #123: Add support for struct field based filtering

rdblue commented on a change in pull request #123: Add support for struct field based filtering
URL: https://github.com/apache/incubator-iceberg/pull/123#discussion_r271456078
 
 

 ##########
 File path: api/src/main/java/org/apache/iceberg/expressions/BoundReference.java
 ##########
 @@ -19,57 +19,238 @@
 
 package org.apache.iceberg.expressions;
 
+import java.io.Serializable;
 import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.Maps;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.StructLike;
 import org.apache.iceberg.exceptions.ValidationException;
 import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
 import org.apache.iceberg.types.Types;
 
 public class BoundReference<T> implements Reference {
   private final int fieldId;
-  private final Type type;
+  private final Accessor<StructLike> accessor;
   private final int pos;
 
-  BoundReference(Types.StructType struct, int fieldId) {
+  BoundReference(Schema schema, int fieldId) {
     this.fieldId = fieldId;
-    this.pos = find(fieldId, struct);
-    this.type = struct.fields().get(pos).type();
+
+    Map<Integer, Accessor<StructLike>> accessors = lazyIdToAccessor(schema);
+
+    this.accessor = accessors.get(fieldId);
+
+    // only look for top level field position
+    this.pos = findTopFieldPos(fieldId, schema.asStruct());
+
   }
 
-  private int find(int fieldId, Types.StructType struct) {
+
+  private int findTopFieldPos(int fieldId, Types.StructType struct) {
     List<Types.NestedField> fields = struct.fields();
     for (int i = 0; i < fields.size(); i += 1) {
       if (fields.get(i).fieldId() == fieldId) {
         return i;
       }
     }
-    throw new ValidationException(
-        "Cannot find top-level field id %d in struct: %s", fieldId, struct);
+    return -1;
   }
 
   public Type type() {
-    return type;
+    return accessor.type();
   }
 
   public int fieldId() {
     return fieldId;
   }
 
   public int pos() {
+    if (pos == -1) {
+      throw new ValidationException("Cannot find position for non-primitive field id %d", fieldId);
+    }
     return pos;
   }
 
   public T get(StructLike struct) {
-    return struct.get(pos, javaType());
+    return (T) accessor.get(struct);
 
 Review comment:
   I think this cast requires `@SuppressWarnings("unchecked")` right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org