beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-4076) Schema followups
Date Fri, 29 Jun 2018 04:08:00 GMT

     [ https://issues.apache.org/jira/browse/BEAM-4076?focusedWorklogId=117192&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-117192
]

ASF GitHub Bot logged work on BEAM-4076:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 29/Jun/18 04:07
            Start Date: 29/Jun/18 04:07
    Worklog Time Spent: 10m 
      Work Description: reuvenlax commented on a change in pull request #5545: [BEAM-4076]
Import Schema branch into master
URL: https://github.com/apache/beam/pull/5545#discussion_r199048533
 
 

 ##########
 File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldAccessDescriptor.java
 ##########
 @@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.beam.sdk.schemas;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.TypeName;
+
+/**
+ * Used inside of a {@link org.apache.beam.sdk.transforms.DoFn} to describe which fields
in a schema
+ * type need to be accessed for processing.
+ */
+@Experimental(Kind.SCHEMAS)
+public class FieldAccessDescriptor implements Serializable {
+  private boolean allFields;
+  private Set<Integer> fieldIdsAccessed;
+  private Set<String> fieldNamesAccessed;
+  private Map<Integer, FieldAccessDescriptor> nestedFieldsAccessedById;
+  private Map<String, FieldAccessDescriptor> nestedFieldsAccessedByName;
+
+  FieldAccessDescriptor(
+      boolean allFields,
+      Set<Integer> fieldsIdsAccessed,
+      Set<String> fieldNamesAccessed,
+      Map<Integer, FieldAccessDescriptor> nestedFieldsAccessedById,
+      Map<String, FieldAccessDescriptor> nestedFieldsAccessedByName) {
+    this.allFields = allFields;
+    this.fieldIdsAccessed = fieldsIdsAccessed;
+    this.fieldNamesAccessed = fieldNamesAccessed;
+    this.nestedFieldsAccessedById = nestedFieldsAccessedById;
+    this.nestedFieldsAccessedByName = nestedFieldsAccessedByName;
+  }
+
+  // Return a descriptor that accesses all fields in a row.
+  public static FieldAccessDescriptor withAllFields() {
+    return new FieldAccessDescriptor(
+        true,
+        Collections.emptySet(),
+        Collections.emptySet(),
+        Collections.emptyMap(),
+        Collections.emptyMap());
+  }
+
+  /**
+   * Return a descriptor that access the specified fields.
+   *
+   * <p>By default, if the field is a nested row (or a container containing a row),
all fields of
+   * said rows are accessed. For finer-grained acccess to nested rows, call withNestedField
and pass
+   * in a recursive {@link FieldAccessDescriptor}.
+   */
+  public static FieldAccessDescriptor withFieldNames(String... names) {
+    return withFieldNames(Arrays.asList(names));
+  }
+
+  /**
+   * Return a descriptor that access the specified fields.
+   *
+   * <p>By default, if the field is a nested row (or a container containing a row),
all fields of
+   * said rows are accessed. For finer-grained acccess to nested rows, call withNestedField
and pass
+   * in a recursive {@link FieldAccessDescriptor}.
+   */
+  public static FieldAccessDescriptor withFieldNames(Iterable<String> fieldNames) {
+    return new FieldAccessDescriptor(
+        false,
+        Collections.emptySet(),
+        Sets.newHashSet(fieldNames),
+        Collections.emptyMap(),
+        Collections.emptyMap());
+  }
+
+  /**
+   * Return a descriptor that access the specified fields.
+   *
+   * <p>By default, if the field is a nested row (or a container containing a row),
all fields of
+   * said rows are accessed. For finer-grained acccess to nested rows, call withNestedField
and pass
+   * in a recursive {@link FieldAccessDescriptor}.
+   */
+  public static FieldAccessDescriptor withFieldIds(Integer... ids) {
+    return withFieldIds(Arrays.asList(ids));
+  }
+
+  /**
+   * Return a descriptor that access the specified fields.
+   *
+   * <p>By default, if the field is a nested row (or a container containing a row),
all fields of
+   * said rows are accessed. For finer-grained acccess to nested rows, call withNestedField
and pass
+   * in a recursive {@link FieldAccessDescriptor}.
+   */
+  public static FieldAccessDescriptor withFieldIds(Iterable<Integer> ids) {
+    return new FieldAccessDescriptor(
+        false,
+        Sets.newHashSet(ids),
+        Collections.emptySet(),
+        Collections.emptyMap(),
+        Collections.emptyMap());
+  }
+
+  /**
+   * Return a descriptor that access the specified nested field. The nested field must be
of type
+   * {@link Schema.TypeName#ROW}, and the fieldAccess argument specifies what fields of the
nested
+   * type will be accessed.
+   */
+  public FieldAccessDescriptor withNestedField(
+      int nestedFieldId, FieldAccessDescriptor fieldAccess) {
+    Map<Integer, FieldAccessDescriptor> newNestedFieldAccess =
+        ImmutableMap.<Integer, FieldAccessDescriptor>builder()
+            .putAll(nestedFieldsAccessedById)
+            .put(nestedFieldId, fieldAccess)
+            .build();
+    return new FieldAccessDescriptor(
+        false,
+        fieldIdsAccessed,
+        fieldNamesAccessed,
+        newNestedFieldAccess,
+        nestedFieldsAccessedByName);
+  }
+
+  /**
+   * Return a descriptor that access the specified nested field. The nested field must be
of type
+   * {@link Schema.TypeName#ROW}, and the fieldAccess argument specifies what fields of the
nested
+   * type will be accessed.
+   */
+  public FieldAccessDescriptor withNestedField(
+      String nestedFieldName, FieldAccessDescriptor fieldAccess) {
+    Map<String, FieldAccessDescriptor> newNestedFieldAccess =
+        ImmutableMap.<String, FieldAccessDescriptor>builder()
+            .putAll(nestedFieldsAccessedByName)
+            .put(nestedFieldName, fieldAccess)
+            .build();
+    return new FieldAccessDescriptor(
+        false,
+        fieldIdsAccessed,
+        fieldNamesAccessed,
+        nestedFieldsAccessedById,
+        newNestedFieldAccess);
+  }
+
+  public boolean allFields() {
+    return allFields;
+  }
+
+  public Set<Integer> fieldIdsAccessed() {
+    return fieldIdsAccessed;
+  }
+
+  public Map<Integer, FieldAccessDescriptor> nestedFields() {
+    return nestedFieldsAccessedById;
+  }
+
+  public FieldAccessDescriptor resolve(Schema schema) {
+    Set<Integer> fieldIdsAccessed = resolveFieldIdsAccessed(schema);
 
 Review comment:
   Done

----------------------------------------------------------------
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:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 117192)
    Time Spent: 6.5h  (was: 6h 20m)

> Schema followups
> ----------------
>
>                 Key: BEAM-4076
>                 URL: https://issues.apache.org/jira/browse/BEAM-4076
>             Project: Beam
>          Issue Type: Improvement
>          Components: beam-model, dsl-sql, sdk-java-core
>            Reporter: Kenneth Knowles
>            Priority: Major
>          Time Spent: 6.5h
>  Remaining Estimate: 0h
>
> This umbrella bug contains subtasks with followups for Beam schemas, which were moved
from SQL to the core Java SDK and made to be type-name-based rather than coder based.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message