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

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

                Author: ASF GitHub Bot
            Created on: 25/Sep/18 13:37
            Start Date: 25/Sep/18 13:37
    Worklog Time Spent: 10m 
      Work Description: akedin commented on a change in pull request #6298: 
[BEAM-4461] Introduce Group transform.
URL: https://github.com/apache/beam/pull/6298#discussion_r220192805
 
 

 ##########
 File path: 
sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
 ##########
 @@ -0,0 +1,684 @@
+/*
+ * 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.transforms;
+
+import java.util.List;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.transforms.Combine.CombineFn;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.sdk.transforms.WithKeys;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * A generic grouping transform for schema {@link PCollection}s.
+ *
+ * <p>When used without a combiner, this transforms simply acts as a {@link 
GroupByKey} but without
+ * the need for the user to explicitly extract the keys. For example, consider 
the following input
+ * type:
+ *
+ * <pre>{@code
+ * {@literal @DefaultSchema(JavaFieldSchema.class)}
+ * public class UserPurchase {
+ *   public String userId;
+ *   public String country;
+ *   public long cost;
+ *   public double transactionDuration;
+ * }
+ *
+ * {@literal PCollection<UserPurchase>} purchases = readUserPurchases();
+ * }</pre>
+ *
+ * <p>You can group all purchases by user and country as follows:
+ *
+ * <pre>{@code
+ * {@literal @DefaultSchema(JavaFieldSchema.class)}
+ * {@literal PCollection<KV<Row, Iterable<UserPurchase>>} byUser =
+ *   purchases.apply(Group.byFieldNames("userId', "country"));
+ * }</pre>
+ *
+ * <p>However often an aggregation of some form is desired. The builder 
methods inside the Group
+ * class allows building up separate aggregations for every field (or set of 
fields) on the input
+ * schema, and generating an output schema based on these aggregations. For 
example:
+ *
+ * <pre>{@code
+ * {@literal PCollection<KV<Row, Row>>} aggregated = purchases
+ *      .apply(Group.byFieldNames("userId', "country")
+ *          .aggregateField("cost", Sum.ofLongs(), "total_cost")
+ *          .aggregateField("cost", Top.<Long>largestLongsFn(10), 
"top_purchases")
+ *          .aggregateField("cost", ApproximateQuantilesCombineFn.create(21),
+ *            Field.of("transactionDurations", 
FieldType.array(FieldType.INT64)));
+ * }</pre>
+ *
+ * <p>The result will be a new row schema containing the fields total_cost, 
top_purchases, and
+ * transactionDurations, containing the sum of all purchases costs (for that 
user and country), the
+ * top ten purchases, and a histogram of transaction durations.
+ *
+ * <p>Note that usually the field type can be automatically inferred from the 
{@link CombineFn}
+ * passed in. However sometimes it cannot be inferred, due to Java type 
erasure, in which case a
+ * {@link Field} object containing the field type must be passed in. This is 
currently the case for
+ * ApproximateQuantilesCombineFn in the above example.
+ */
+@Experimental(Kind.SCHEMAS)
+public class Group {
+  /**
+   * Returns a transform that groups all elements in the input {@link 
PCollection}. The returned
+   * transform contains further builder methods to control how the grouping is 
done.
+   */
+  public static <T> Global<T> globally() {
+    return new Global<>();
+  }
+
+  /**
+   * Returns a transform that groups all elements in the input {@link 
PCollection} keyed by the list
+   * of fields specified. The output of this transform will be a {@link KV} 
keyed by a {@link Row}
+   * containing the specified extracted fields. The returned transform 
contains further builder
+   * methods to control how the grouping is done.
+   */
+  public static <T> ByFields<T> byFieldNames(String... fieldNames) {
+    return new ByFields<>(FieldAccessDescriptor.withFieldNames(fieldNames));
+  }
+
+  /**
+   * Returns a transform that groups all elements in the input {@link 
PCollection} keyed by the list
+   * of fields specified. The output of this transform will be a {@link KV} 
keyed by a {@link Row}
+   * containing the specified extracted fields. The returned transform 
contains further builder
+   * methods to control how the grouping is done.
+   */
+  public static <T> ByFields<T> byFieldIds(Integer... fieldIds) {
 
 Review comment:
   Do we need to expose access by field ids? If there's no technical need I'd 
rather avoid this. In SQL we did expose access by field but it end up not used 
anywhere.

----------------------------------------------------------------
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: 147563)
    Time Spent: 11h  (was: 10h 50m)

> Create a library of useful transforms that use schemas
> ------------------------------------------------------
>
>                 Key: BEAM-4461
>                 URL: https://issues.apache.org/jira/browse/BEAM-4461
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-java-core
>            Reporter: Reuven Lax
>            Assignee: Reuven Lax
>            Priority: Major
>          Time Spent: 11h
>  Remaining Estimate: 0h
>
> e.g. JoinBy(fields). Project, Filter, etc.



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

Reply via email to