You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2021/02/18 23:04:00 UTC

[jira] [Work logged] (BEAM-11800) Support ARRAY_AGG fn for Zetasql dialect

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

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

                Author: ASF GitHub Bot
            Created on: 18/Feb/21 23:03
            Start Date: 18/Feb/21 23:03
    Worklog Time Spent: 10m 
      Work Description: robinyqiu commented on a change in pull request #13483:
URL: https://github.com/apache/beam/pull/13483#discussion_r578808090



##########
File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udaf/ArrayAgg.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.extensions.sql.impl.udaf;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.transforms.Combine;
+
+public class ArrayAgg {
+
+  public static class ArrayAggArray extends Combine.CombineFn<Object, List<Object>, Object[]> {
+    @Override
+    public List<Object> createAccumulator() {
+      return new ArrayList<>();
+    }
+
+    @Override
+    public List<Object> addInput(List<Object> accum, Object input) {
+      accum.add(input);
+      return accum;
+    }
+
+    @Override
+    public List<Object> mergeAccumulators(Iterable<List<Object>> accums) {
+      List<Object> merged = new ArrayList<>();
+      for (List<Object> accum : accums) {
+        for (Object o : accum) {
+          merged.add(o);
+        }
+      }
+      return merged;
+    }
+
+    @Override
+    public Object[] extractOutput(List<Object> accumulator) {

Review comment:
       The Beam `ARRAY` type expect the data to be a `Collection` type, not `Object[]` type (i.e Beam `ARRAY` != Java array). That's why you get the error during cast. The fix should be simple: change the return type of this function to list (also the third class generic parameter).




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


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

            Worklog Id:     (was: 554515)
    Remaining Estimate: 0h
            Time Spent: 10m

> Support ARRAY_AGG fn for Zetasql dialect
> ----------------------------------------
>
>                 Key: BEAM-11800
>                 URL: https://issues.apache.org/jira/browse/BEAM-11800
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql-zetasql
>            Reporter: Sonam Ramchand
>            Priority: P2
>          Time Spent: 10m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)