[ https://issues.apache.org/jira/browse/BEAM-8365?focusedWorklogId=328731&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-328731 ]
ASF GitHub Bot logged work on BEAM-8365: ---------------------------------------- Author: ASF GitHub Bot Created on: 15/Oct/19 18:26 Start Date: 15/Oct/19 18:26 Worklog Time Spent: 10m Work Description: apilloud commented on pull request #9764: [BEAM-8365] Project push-down for TestTableProvider URL: https://github.com/apache/beam/pull/9764#discussion_r335099406 ########## File path: sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java ########## @@ -0,0 +1,236 @@ +/* + * 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.rule; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Queue; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSourceRel; +import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor; +import org.apache.beam.sdk.schemas.FieldAccessDescriptor.FieldDescriptor; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.utils.SelectHelpers; +import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelRecordType; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilder; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory; +import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting; + +public class BeamIOPushDownRule extends RelOptRule { + // ~ Static fields/initializers --------------------------------------------- + + public static final BeamIOPushDownRule INSTANCE = + new BeamIOPushDownRule(RelFactories.LOGICAL_BUILDER); + + // ~ Constructors ----------------------------------------------------------- + + public BeamIOPushDownRule(RelBuilderFactory relBuilderFactory) { + super(operand(Calc.class, operand(BeamIOSourceRel.class, any())), relBuilderFactory, null); + } + + // ~ Methods ---------------------------------------------------------------- + + @Override + public void onMatch(RelOptRuleCall call) { + final Calc calc = call.rel(0); + final BeamIOSourceRel ioSourceRel = call.rel(1); + final BeamSqlTable beamSqlTable = ioSourceRel.getBeamSqlTable(); + final RexProgram program = calc.getProgram(); + final Pair<ImmutableList<RexNode>, ImmutableList<RexNode>> projectFilter = program.split(); + final RelDataType calcInputRowType = program.getInputRowType(); + RelBuilder relBuilder = call.builder(); + + if (!beamSqlTable.supportsProjects()) { + return; + } + + // Nested rows are not supported at the moment + for (RelDataTypeField field : ioSourceRel.getRowType().getFieldList()) { + if (field.getType() instanceof RelRecordType) { + return; + } + } + + // Find all input refs used by projects + Set<String> usedFields = new HashSet<>(); + for (RexNode project : projectFilter.left) { + findUtilizedInputRefs(calcInputRowType, project, usedFields); + } + + // Check if the calc can be dropped + if (isProjectRenameOnlyProgram(program)) { + call.transformTo(ioSourceRel.copy(calc.getRowType(), new ArrayList<>(usedFields))); + return; + } + + // Find all input refs used by filters + for (RexNode filter : projectFilter.right) { + findUtilizedInputRefs(calcInputRowType, filter, usedFields); + } + + // TODO: will need to be updated for predicate push-down + if (usedFields.size() + == ioSourceRel.getRowType().getFieldCount()) { // Already most optimal case + return; + } + + FieldAccessDescriptor resolved = + FieldAccessDescriptor.withFieldNames(usedFields).resolve(beamSqlTable.getSchema()); + Schema newSchema = + SelectHelpers.getOutputSchema(ioSourceRel.getBeamSqlTable().getSchema(), resolved); + RelDataType calcInputType = + CalciteUtils.toCalciteRowType(newSchema, ioSourceRel.getCluster().getTypeFactory()); + + BeamIOSourceRel newIoSourceRel = ioSourceRel.copy(calcInputType, new ArrayList<>(usedFields)); Review comment: Just converting to an `ArrayList` is a little scary to me. Is this list guaranteed to be in the same order as the output row? (I'm guessing there are IOs that would expect it to be.) ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 328731) Time Spent: 4h 50m (was: 4h 40m) > Add project push-down capability to IO APIs > ------------------------------------------- > > Key: BEAM-8365 > URL: https://issues.apache.org/jira/browse/BEAM-8365 > Project: Beam > Issue Type: New Feature > Components: dsl-sql > Reporter: Kirill Kozlov > Assignee: Kirill Kozlov > Priority: Major > Time Spent: 4h 50m > Remaining Estimate: 0h > > * InMemoryTable should implement a following method: > {code:java} > public PCollection<Row> buildIOReader( > PBegin begin, BeamSqlTableFilter filters, List<String> fieldNames);{code} > Which should return a `PCollection` with fields specified in `fieldNames` > list. > * Create a rule to push fields used by a Calc (in projects and in a > condition) down into TestTable IO. > * Updating that same Calc (from previous step) to have a proper input and > output schemes, remove unused fields. -- This message was sent by Atlassian Jira (v8.3.4#803005)