[
https://issues.apache.org/jira/browse/BEAM-4167?focusedWorklogId=104912&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-104912
]
ASF GitHub Bot logged work on BEAM-4167:
----------------------------------------
Author: ASF GitHub Bot
Created on: 23/May/18 00:46
Start Date: 23/May/18 00:46
Worklog Time Spent: 10m
Work Description: kennknowles commented on a change in pull request
#5428: [BEAM-4167] Implement UNNEST
URL: https://github.com/apache/beam/pull/5428#discussion_r190095340
##########
File path:
sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.rel;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.List;
+import
org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlExpressionExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.interpreter.BeamSqlFnExecutor;
+import org.apache.beam.sdk.extensions.sql.impl.schema.BeamTableUtils;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.Row;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Uncollect;
+import org.apache.calcite.sql.SemiJoinType;
+import org.apache.calcite.util.ImmutableBitSet;
+
+/**
+ * {@link BeamRelNode} to implement UNNEST, supporting specifically only
{@link Correlate} with
+ * {@link Uncollect}.
+ */
+public class BeamUnnestRel extends Correlate implements BeamRelNode {
+
+ public BeamUnnestRel(
+ RelOptCluster cluster,
+ RelTraitSet traits,
+ RelNode left,
+ RelNode right,
+ CorrelationId correlationId,
+ ImmutableBitSet requiredColumns,
+ SemiJoinType joinType) {
+ super(cluster, traits, left, right, correlationId, requiredColumns,
joinType);
+ }
+
+ @Override
+ public PTransform<PCollectionTuple, PCollection<Row>> toPTransform() {
+ return new Transform();
+ }
+
+ @Override
+ public Correlate copy(
+ RelTraitSet relTraitSet,
+ RelNode left,
+ RelNode right,
+ CorrelationId correlationId,
+ ImmutableBitSet requireColumns,
+ SemiJoinType joinType) {
+ return new BeamUnnestRel(
+ getCluster(), relTraitSet, left, right, correlationId,
requiredColumns, joinType);
+ }
+
+ private class Transform extends PTransform<PCollectionTuple,
PCollection<Row>> {
+ @Override
+ public PCollection<Row> expand(PCollectionTuple inputPCollections) {
+ String stageName = BeamSqlRelUtils.getStageName(BeamUnnestRel.this);
+
+ // The set of rows where we run the correlated unnest for each row
+ PCollection<Row> outer =
+
inputPCollections.apply(BeamSqlRelUtils.getBeamRelInput(left).toPTransform());
+
+ // The correlated subquery
+ BeamUncollectRel uncollect = (BeamUncollectRel)
BeamSqlRelUtils.getBeamRelInput(right);
+ Schema innerSchema = CalciteUtils.toBeamSchema(uncollect.getRowType());
+ checkArgument(
+ innerSchema.getFieldCount() == 1, "Can only UNNEST a single column",
getClass());
+
+ BeamSqlExpressionExecutor expr =
+ new
BeamSqlFnExecutor(BeamSqlRelUtils.getBeamRelInput(uncollect.getInput()));
+
+ Schema joinedSchema = CalciteUtils.toBeamSchema(rowType);
+
+ return outer
+ .apply(
+ stageName,
+ ParDo.of(
+ new UnnestFn(correlationId.getId(), expr, joinedSchema,
innerSchema.getField(0))))
+ .setCoder(joinedSchema.getRowCoder());
+ }
+ }
+
+ private static class UnnestFn extends DoFn<Row, Row> {
+
+ /** The expression that should return an iterable to be uncollected. */
+ private final BeamSqlExpressionExecutor expr;
+ private final int correlationId;
+ private final Schema outputSchema;
+ private final Schema.Field innerField;
+
+ private UnnestFn(
+ int correlationId, BeamSqlExpressionExecutor expr, Schema
outputSchema, Schema.Field innerField) {
+ this.correlationId = correlationId;
+ this.expr = expr;
+ this.outputSchema = outputSchema;
+ this.innerField = innerField;
+ }
+
+ @ProcessElement
+ public void process(@Element Row row, BoundedWindow window,
OutputReceiver<Row> out) {
+
+ List<Object> rawValues = expr.execute(row, window,
ImmutableMap.of(correlationId, row));
+
+ checkState(
+ rawValues.size() == 1,
+ "%s expression to unnest %s resulted in more than one column",
+ getClass(),
+ expr);
+
+ checkState(
+ rawValues.get(0) instanceof Iterable,
+ "%s expression to unnest %s not iterable",
+ getClass(),
+ expr);
+
+ for (Object uncollectedValue : (Iterable) rawValues.get(0)) {
Review comment:
@akedin this line is where the `MULTISET` value is used.
----------------------------------------------------------------
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: 104912)
Time Spent: 2.5h (was: 2h 20m)
> Implement UNNEST
> ----------------
>
> Key: BEAM-4167
> URL: https://issues.apache.org/jira/browse/BEAM-4167
> Project: Beam
> Issue Type: New Feature
> Components: dsl-sql
> Reporter: Anton Kedin
> Assignee: Kenneth Knowles
> Priority: Major
> Time Spent: 2.5h
> Remaining Estimate: 0h
>
> We need to be able to convert collections to relations in the query to
> perform any meaningful operations on them.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)