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

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

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

 ##########
 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")
 
 Review comment:
   This is closer to the Dataframes model, which I think is what we should be 
aiming at for a programmatic approach. It's also what fits in with Beam, where 
aggregations and groupbykey are intimately related. I don't think these 
transforms have to match SQL - we already have SQL for that scenario.
   
   The output schema is specified by the aggregateField calls (that's why you 
have to specify a field name in those calls). 

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

> 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 20m
>  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