From 57fddbd8e3705254a52e5e478a0dcf9493142203 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Wed, 8 Mar 2017 21:09:03 -0800 Subject: [PATCH 01/15] save work: BaseBeamTable, BeamRelNode, BeamRule --- sdks/java/pom.xml | 6 +- sdks/java/sql/pom.xml | 139 ++++++++++++++++++ .../sdk/java/sql/examples/BeamSqlRunner.java | 103 +++++++++++++ .../sdk/java/sql/examples/package-info.java | 22 +++ .../BeamSQLExpressionExecutor.java | 7 + .../java/sql/interpreter/package-info.java | 8 + .../org/beam/sdk/java/sql/package-info.java | 22 +++ .../java/sql/planner/BeamPipelineCreator.java | 73 +++++++++ .../java/sql/planner/BeamQueryPlanner.java | 119 +++++++++++++++ .../sql/planner/BeamRelDataTypeSystem.java | 23 +++ .../sdk/java/sql/planner/BeamRuleSets.java | 64 ++++++++ .../planner/BeamSqlUnsupportedException.java | 34 +++++ .../planner/UnsupportedOperatorsVisitor.java | 24 +++ .../sdk/java/sql/planner/package-info.java | 22 +++ .../beam/sdk/java/sql/rel/BeamFilterRel.java | 35 +++++ .../beam/sdk/java/sql/rel/BeamIOSinkRel.java | 41 ++++++ .../sdk/java/sql/rel/BeamIOSourceRel.java | 30 ++++ .../java/sql/rel/BeamLogicalConvention.java | 68 +++++++++ .../beam/sdk/java/sql/rel/BeamProjectRel.java | 46 ++++++ .../beam/sdk/java/sql/rel/BeamRelNode.java | 18 +++ .../beam/sdk/java/sql/rel/package-info.java | 8 + .../sdk/java/sql/rule/BeamFilterRule.java | 28 ++++ .../sdk/java/sql/rule/BeamIOSinkRule.java | 59 ++++++++ .../sdk/java/sql/rule/BeamIOSourceRule.java | 27 ++++ .../sdk/java/sql/rule/BeamProjectRule.java | 46 ++++++ .../beam/sdk/java/sql/rule/package-info.java | 8 + .../sdk/java/sql/schema/BaseBeamTable.java | 97 ++++++++++++ .../beam/sdk/java/sql/schema/BeamIOType.java | 11 ++ .../java/sql/schema/BeamSQLRecordType.java | 51 +++++++ .../beam/sdk/java/sql/schema/BeamSQLRow.java | 114 ++++++++++++++ .../java/sql/schema/kafka/BeamKafkaTable.java | 98 ++++++++++++ .../schema/kafka/KafkaCSVRecordTransform.java | 40 +++++ .../java/sql/schema/kafka/package-info.java | 8 + .../sdk/java/sql/schema/package-info.java | 22 +++ .../java/sql/transform/BeamSQLFilterFn.java | 47 ++++++ .../transform/BeamSQLOutputToConsoleFn.java | 24 +++ .../java/sql/transform/BeamSQLProjectFn.java | 76 ++++++++++ .../sdk/java/sql/transform/CalciteToSpEL.java | 65 ++++++++ .../sdk/java/sql/transform/ProjectRule.java | 48 ++++++ .../sdk/java/sql/transform/ProjectType.java | 7 + .../sdk/java/sql/transform/package-info.java | 8 + 41 files changed, 1793 insertions(+), 3 deletions(-) create mode 100644 sdks/java/sql/pom.xml create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlRunner.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/CalciteToSpEL.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectRule.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectType.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index 7ca61090e7b2..eb07bfe0c362 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -14,8 +14,7 @@ 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. ---> - +--> 4.0.0 @@ -43,6 +42,7 @@ + sql @@ -57,4 +57,4 @@ - + \ No newline at end of file diff --git a/sdks/java/sql/pom.xml b/sdks/java/sql/pom.xml new file mode 100644 index 000000000000..9c29a450f6d2 --- /dev/null +++ b/sdks/java/sql/pom.xml @@ -0,0 +1,139 @@ + + + 4.0.0 + + org.apache.beam + beam-sdks-java-parent + 0.6.0-SNAPSHOT + + + beam-sdk-java-sql + Apache Beam :: SDKs :: Java :: SQL + Beam SQL provides a new interface to generate a Beam pipeline from SQL statement + + jar + + + ${maven.build.timestamp} + yyyy-MM-dd HH:mm + 1.11.0 + + + + + + src/main/resources + true + + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + ${project.basedir}/src/test/ + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.apache.beam.sdk.testing.NeedsRunner + + + true + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + org.apache.maven.plugins + maven-shade-plugin + + + bundle-and-repackage + package + + shade + + + true + + + com.google.guava:guava + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + org.jacoco + jacoco-maven-plugin + + + + + + + junit + junit + + + org.apache.calcite + calcite-core + ${calcite-version} + + + org.apache.beam + beam-sdks-java-core + + + org.apache.beam + beam-runners-direct-java + + + org.apache.beam + beam-sdks-java-io-kafka + + + org.springframework + spring-expression + + + com.google.guava + guava + + + diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlRunner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlRunner.java new file mode 100644 index 000000000000..b17d6801e25c --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlRunner.java @@ -0,0 +1,103 @@ +package org.beam.sdk.java.sql.examples; + +import java.io.IOException; +import java.sql.SQLException; + +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.RelFieldCollation; +import org.apache.calcite.rel.RelFieldCollation.Direction; +import org.apache.calcite.rel.RelFieldCollation.NullDirection; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.schema.Schema; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Statistic; +import org.apache.calcite.schema.Statistics; +import org.apache.calcite.schema.Table; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.util.ImmutableBitSet; +import org.beam.sdk.java.sql.planner.BeamQueryPlanner; + +import com.google.common.collect.ImmutableList; + +public class BeamSqlRunner { + private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + private final SchemaPlus schema = Frameworks.createRootSchema(true); + + public void addSchema(String schemaName, Schema scheme) { + schema.add(schemaName, schema); + } + + public void addTable(String tableName, Table table) { + schema.add(tableName, table); + } + + public void explainAndRun(String sqlString) throws IOException, SQLException { + + BeamQueryPlanner planner = new BeamQueryPlanner(schema); + try { + System.out.println("SQL>: \n" + sqlString); + planner.compileAndRun(sqlString); + } catch (Exception e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + } + + public static void main(String[] args) throws IOException, SQLException { + BeamSqlRunner runner = new BeamSqlRunner(); + runner.initTables(); + + // case 2: insert into () select STREAM from + //
from + String sql = "SELECT " + " SITEID, PAGEID as new_pageId,floor(EVENTTIMESTAMP TO HOUR) AS EVENT_HOUR" + + ", COUNT(*) AS SIZE " + "FROM SOJ_EVENT " + + "WHERE SITEID >= 0 " + "GROUP BY SITEID, PAGEID, floor(EVENTTIMESTAMP TO HOUR)"; + + sql = "SELECT " + " SITEID, PAGEID, HOP_START(EVENTTIMESTAMP, INTERVAL '1' HOUR, INTERVAL '3' HOUR) " + + ", COUNT(*) AS SIZE " + "FROM SOJ_EVENT " + + "WHERE SITEID >= 0 " + "GROUP BY SITEID, PAGEID, HOP(EVENTTIMESTAMP, INTERVAL '1' HOUR, INTERVAL '3' HOUR) "; + + runner.explainAndRun(sql); + } + + void initTables() { + addTable("SOJ_EVENT", + "rheos-kafka-proxy-1.lvs02.dev.ebayc3.com:9092,rheos-kafka-proxy-2.lvs02.dev.ebayc3.com:9092", + "behavior.pulsar.sojevent.total"); + addTable("subrowverevent", "flink2-8332.lvs01.dev.ebayc3.com:9092", "subrowverevent"); + // addTable("externalorder", "flink2-8332.lvs01.dev.ebayc3.com:9092", + // "externalorder"); + } + + void addTable(String tableName, String bootstrapServer, String topic) { + RelProtoDataType protoRowType = new RelProtoDataType() { + @Override + public RelDataType apply(RelDataTypeFactory a0) { + return a0.builder().add("EVENTTIMESTAMP", SqlTypeName.TIMESTAMP) + .add("ITEMID", SqlTypeName.BIGINT).add("SITEID", SqlTypeName.INTEGER) + .add("PAGEID", SqlTypeName.INTEGER).add("PAGENAME", SqlTypeName.VARCHAR) + // .add("APPLICATIONPAYLOAD", SqlTypeName.INTEGER) + .build(); + } + }; + Direction dir = Direction.ASCENDING; + RelFieldCollation collation = new RelFieldCollation(0, dir, NullDirection.UNSPECIFIED); + Statistic stat = Statistics.of(5, ImmutableList.of(ImmutableBitSet.of(0)), + ImmutableList.of(RelCollations.of(collation))); + + stat = Statistics.of(100d, + ImmutableList.of(), +RelCollations.createSingleton(0)); + + stat = Statistics.UNKNOWN; + +// addTable(tableName, +// new BeamKafkaTable(protoRowType, stat, bootstrapServer, topic)); + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java new file mode 100644 index 000000000000..f42eaaf55597 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * Contains some examples on how to use BeamSQL to simplify pipeline assembling. + * + */ +package org.beam.sdk.java.sql.examples; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java new file mode 100644 index 000000000000..fb57bfd02f99 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java @@ -0,0 +1,7 @@ +package org.beam.sdk.java.sql.interpreter; + +import java.io.Serializable; + +public interface BeamSQLExpressionExecutor extends Serializable { + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java new file mode 100644 index 000000000000..244b243ea618 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java @@ -0,0 +1,8 @@ +/** + * + */ +/** + * interpreter generate runnable 'code' to execute SQL operation, expressions. + * + */ +package org.beam.sdk.java.sql.interpreter; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java new file mode 100644 index 000000000000..007ee431af67 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * BeamSQL provides a new interface to generate Beam pipeline from a SQL statement. + * + */ +package org.beam.sdk.java.sql; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java new file mode 100644 index 000000000000..1273b4e7be29 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java @@ -0,0 +1,73 @@ +package org.beam.sdk.java.sql.planner; + +import java.util.Map; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.beam.sdk.java.sql.rel.BeamRelNode; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.sdk.java.sql.transform.BeamSQLOutputToConsoleFn; +import org.joda.time.Duration; + +/** + * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam pipeline. + * + */ +public class BeamPipelineCreator { + private Map sourceTables; + private PCollection latestStream; + + private PipelineOptions options; + + private Pipeline pipeline; + + private boolean hasPersistent = false; + + public BeamPipelineCreator(Map sourceTables) { + this.sourceTables = sourceTables; + + options = PipelineOptionsFactory.fromArgs(new String[] {}).withValidation() + .as(PipelineOptions.class); // FlinkPipelineOptions.class + options.setJobName("BeamPlanCreator"); + + pipeline = Pipeline.create(options); + } + + public void runJob() { + if (!hasPersistent) { + latestStream.apply("emit_to_console", ParDo.of(new BeamSQLOutputToConsoleFn("emit_to_console"))); + } + + pipeline.run(); + } + + public PCollection getLatestStream() { + return latestStream; + } + + public void setLatestStream(PCollection latestStream) { + this.latestStream = latestStream; + } + + public Map getKafkaTables() { + return sourceTables; + } + + public Pipeline getPipeline() { + return pipeline; + } + + public boolean isHasPersistent() { + return hasPersistent; + } + + public void setHasPersistent(boolean hasPersistent) { + this.hasPersistent = hasPersistent; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java new file mode 100644 index 000000000000..7e83c40d5add --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java @@ -0,0 +1,119 @@ +package org.beam.sdk.java.sql.planner; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.config.Lex; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.plan.Contexts; +import org.apache.calcite.plan.ConventionTraitDef; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.plan.RelTraitDef; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.tools.FrameworkConfig; +import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.tools.Planner; +import org.apache.calcite.tools.RelConversionException; +import org.apache.calcite.tools.ValidationException; +import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +import org.beam.sdk.java.sql.rel.BeamRelNode; +import org.beam.sdk.java.sql.schema.BaseBeamTable; + +/** + * The core component to handle through a SQL statement, to a Beam pipeline. + * + */ +public class BeamQueryPlanner { + + public static final int BEAM_REL_CONVERSION_RULES = 1; + + private final Planner planner; + private Map kafkaTables = new HashMap<>(); + + private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + + public BeamQueryPlanner(SchemaPlus schema) { + final List traitDefs = new ArrayList(); + traitDefs.add(ConventionTraitDef.INSTANCE); + traitDefs.add(RelCollationTraitDef.INSTANCE); + + List sqlOperatorTables = new ArrayList<>(); + sqlOperatorTables.add(SqlStdOperatorTable.instance()); + sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false, + Collections.emptyList(), typeFactory)); + + FrameworkConfig config = Frameworks.newConfigBuilder() + .parserConfig(SqlParser.configBuilder() + .setLex(Lex.MYSQL).build()) + .defaultSchema(schema) + .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets()) + .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build(); + this.planner = Frameworks.getPlanner(config); + + for (String t : schema.getTableNames()) { + kafkaTables.put(t, (BaseBeamTable) schema.getTable(t)); + } + } + + public void compileAndRun(String query) throws Exception { + BeamRelNode relNode = getPlan(query); + + BeamPipelineCreator planCreator = new BeamPipelineCreator(kafkaTables); + + String beamPlan = RelOptUtil.toString(relNode); + System.out.println("beamPlan>"); + System.out.println(beamPlan); + + relNode.buildBeamPipeline(planCreator); + + planCreator.runJob(); + } + + public BeamRelNode getPlan(String query) + throws ValidationException, RelConversionException, SqlParseException { + return (BeamRelNode) validateAndConvert(planner.parse(query)); + } + + private RelNode validateAndConvert(SqlNode sqlNode) + throws ValidationException, RelConversionException { + SqlNode validated = validateNode(sqlNode); + RelNode relNode = convertToRelNode(validated); + return convertToBeamRel(relNode); + } + + private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException { + RelTraitSet traitSet = relNode.getTraitSet(); + // traitSet = traitSet.simplify(); + + System.out.println("SQLPlan>\n" + RelOptUtil.toString(relNode)); + + // PlannerImpl.transform() optimizes RelNode with ruleset + return planner.transform(1, traitSet.replace(BeamLogicalConvention.INSTANCE), relNode); + } + + private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException { + return planner.rel(sqlNode).rel; + } + + private SqlNode validateNode(SqlNode sqlNode) throws ValidationException { + SqlNode validatedSqlNode = planner.validate(sqlNode); + validatedSqlNode.accept(new UnsupportedOperatorsVisitor()); + return validatedSqlNode; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java new file mode 100644 index 000000000000..65dd473c579a --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java @@ -0,0 +1,23 @@ +package org.beam.sdk.java.sql.planner; + +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.rel.type.RelDataTypeSystemImpl; + +/** + * customized data type in Beam. + * + */ +public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl { + public static final RelDataTypeSystem BEAM_REL_DATATYPE_SYSTEM = new BeamRelDataTypeSystem(); + + @Override + public int getMaxNumericScale() { + return 38; + } + + @Override + public int getMaxNumericPrecision() { + return 38; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java new file mode 100644 index 000000000000..646c6ee033c8 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java @@ -0,0 +1,64 @@ +package org.beam.sdk.java.sql.planner; + +import java.util.Iterator; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.rules.CalcMergeRule; +import org.apache.calcite.rel.rules.FilterCalcMergeRule; +import org.apache.calcite.rel.rules.FilterProjectTransposeRule; +import org.apache.calcite.rel.rules.FilterToCalcRule; +import org.apache.calcite.rel.rules.ProjectCalcMergeRule; +import org.apache.calcite.rel.rules.ProjectFilterTransposeRule; +import org.apache.calcite.rel.rules.ProjectRemoveRule; +import org.apache.calcite.rel.rules.ProjectToCalcRule; +import org.apache.calcite.rel.rules.PruneEmptyRules; +import org.apache.calcite.rel.rules.ReduceExpressionsRule; +import org.apache.calcite.rel.rules.SortRemoveRule; +import org.apache.calcite.rel.rules.UnionEliminatorRule; +import org.apache.calcite.rel.stream.StreamRules; +import org.apache.calcite.tools.RuleSet; +import org.beam.sdk.java.sql.rel.BeamRelNode; +import org.beam.sdk.java.sql.rule.BeamFilterRule; +import org.beam.sdk.java.sql.rule.BeamIOSinkRule; +import org.beam.sdk.java.sql.rule.BeamIOSourceRule; +import org.beam.sdk.java.sql.rule.BeamProjectRule; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +/** + * {@link RuleSet} which translate a standard Calcite {@link RelNode} tree, to represent with {@link BeamRelNode} + * + */ +public class BeamRuleSets { + private static final ImmutableSet calciteToBeamConversionRules = ImmutableSet + .builder() + .add( + BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE, BeamFilterRule.INSTANCE, + BeamIOSinkRule.INSTANCE + ).build(); + + public static RuleSet[] getRuleSets() { + return new RuleSet[] {new BeamRuleSet(ImmutableSet.builder() + .addAll(calciteToBeamConversionRules).build()) }; + } + + private static class BeamRuleSet implements RuleSet { + final ImmutableSet rules; + + public BeamRuleSet(ImmutableSet rules) { + this.rules = rules; + } + + public BeamRuleSet(ImmutableList rules) { + this.rules = ImmutableSet.builder().addAll(rules).build(); + } + + @Override + public Iterator iterator() { + return rules.iterator(); + } + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java new file mode 100644 index 000000000000..49ccb12dce5f --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java @@ -0,0 +1,34 @@ +/* + * 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.beam.sdk.java.sql.planner; + +public class BeamSqlUnsupportedException extends RuntimeException { + /** + * + */ + private static final long serialVersionUID = 3445015747629217342L; + + public BeamSqlUnsupportedException(String string) { + super(string); + } + + public BeamSqlUnsupportedException() { + super(); + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java new file mode 100644 index 000000000000..daf720837034 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java @@ -0,0 +1,24 @@ +/* + * 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.beam.sdk.java.sql.planner; + +import org.apache.calcite.sql.util.SqlShuttle; + +public class UnsupportedOperatorsVisitor extends SqlShuttle { + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java new file mode 100644 index 000000000000..652a4f91fef0 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * {@link org.beam.sdk.java.sql.planner.BeamQueryPlanner} is the main interface, to define data sources, validate a SQL statement, and convert it as a Beam pipeline. + * + */ +package org.beam.sdk.java.sql.planner; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java new file mode 100644 index 000000000000..ead31e99f66a --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java @@ -0,0 +1,35 @@ +package org.beam.sdk.java.sql.rel; + +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.beam.sdk.java.sql.planner.BeamPipelineCreator; + +public class BeamFilterRel extends Filter implements BeamRelNode { + + public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, + RexNode condition) { + super(cluster, traits, child, condition); + } + + @Override + public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) { + return new BeamFilterRel(getCluster(), traitSet, input, condition); + } + + @Override + public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + // TODO Auto-generated method stub + + } + + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java new file mode 100644 index 000000000000..3b0508cfb7d2 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java @@ -0,0 +1,41 @@ +package org.beam.sdk.java.sql.rel; + +import java.util.List; + +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.prepare.Prepare; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rex.RexNode; +import org.beam.sdk.java.sql.planner.BeamPipelineCreator; + +public class BeamIOSinkRel extends TableModify implements BeamRelNode { + public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, + Prepare.CatalogReader catalogReader, RelNode child, Operation operation, + List updateColumnList, List sourceExpressionList, boolean flattened) { + super(cluster, traits, table, catalogReader, child, operation, updateColumnList, + sourceExpressionList, flattened); + } + + @Override + public RelNode copy(RelTraitSet traitSet, List inputs) { + return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), + sole(inputs), getOperation(), getUpdateColumnList(), getSourceExpressionList(), + isFlattened()); + } + + @Override + public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + // TODO Auto-generated method stub + + } + + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java new file mode 100644 index 000000000000..8d78acc41d73 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java @@ -0,0 +1,30 @@ +package org.beam.sdk.java.sql.rel; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.core.TableScan; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.beam.sdk.java.sql.planner.BeamPipelineCreator; + +public class BeamIOSourceRel extends TableScan implements BeamRelNode { + + public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) { + super(cluster, traitSet, table); + } + + @Override + public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + // TODO Auto-generated method stub + + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java new file mode 100644 index 000000000000..18d7eaf6bb38 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java @@ -0,0 +1,68 @@ +/** + * 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.beam.sdk.java.sql.rel; + +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.ConventionTraitDef; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelTrait; +import org.apache.calcite.plan.RelTraitDef; +import org.apache.calcite.plan.RelTraitSet; + +public enum BeamLogicalConvention implements Convention { + INSTANCE; + + @Override + public Class getInterface() { + return BeamRelNode.class; + } + + @Override + public String getName() { + return "BEAM_LOGICAL"; + } + + @Override + public RelTraitDef getTraitDef() { + return ConventionTraitDef.INSTANCE; + } + + @Override + public boolean satisfies(RelTrait trait) { + return this == trait; + } + + @Override + public void register(RelOptPlanner planner) { + } + + @Override + public String toString() { + return getName(); + } + + @Override + public boolean canConvertConvention(Convention toConvention) { + return false; + } + + @Override + public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) { + return false; + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java new file mode 100644 index 000000000000..00ede43dfacf --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java @@ -0,0 +1,46 @@ +package org.beam.sdk.java.sql.rel; + +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.beam.sdk.java.sql.planner.BeamPipelineCreator; + +public class BeamProjectRel extends Project implements BeamRelNode { + + /** + * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall} + * + * @param cluster + * @param traits + * @param input + * @param projects + * @param rowType + */ + public BeamProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode input, + List projects, RelDataType rowType) { + super(cluster, traits, input, projects, rowType); + } + + @Override + public Project copy(RelTraitSet traitSet, RelNode input, List projects, + RelDataType rowType) { + return new BeamProjectRel(getCluster(), traitSet, input, projects, rowType); + } + + @Override + public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + // TODO Auto-generated method stub + + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java new file mode 100644 index 000000000000..67c7153f0f32 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java @@ -0,0 +1,18 @@ +package org.beam.sdk.java.sql.rel; + +import org.apache.calcite.rel.RelNode; +import org.beam.sdk.java.sql.planner.BeamPipelineCreator; + +/** + * A new method {@link #buildBeamPipeline(BeamPipelineCreator)} is added, it's called by {@link BeamPipelineCreator}. + * + */ +public interface BeamRelNode extends RelNode { + + /** + * A {@link BeamRelNode} is a recursive structure, the {@link BeamPipelineCreator} visits it with a DFS(Depth-First-Search) algorithm. + * @param planCreator + * @throws Exception + */ + void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception; +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java new file mode 100644 index 000000000000..e1e53d8edbdc --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java @@ -0,0 +1,8 @@ +/** + * + */ +/** + * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode} + * + */ +package org.beam.sdk.java.sql.rel; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java new file mode 100644 index 000000000000..37afbb56c4a7 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java @@ -0,0 +1,28 @@ +package org.beam.sdk.java.sql.rule; + +import org.apache.calcite.plan.Convention; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.beam.sdk.java.sql.rel.BeamFilterRel; +import org.beam.sdk.java.sql.rel.BeamLogicalConvention; + +public class BeamFilterRule extends ConverterRule { + public static BeamFilterRule INSTANCE = new BeamFilterRule(); + + private BeamFilterRule() { + super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final Filter filter = (Filter) rel; + final RelNode input = filter.getInput(); + + return new BeamFilterRel(filter.getCluster(), + filter.getTraitSet().replace(BeamLogicalConvention.INSTANCE), + convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)), + filter.getCondition()); + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java new file mode 100644 index 000000000000..370191ae045a --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java @@ -0,0 +1,59 @@ +package org.beam.sdk.java.sql.rule; + +import java.util.List; + +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.prepare.Prepare; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rel.logical.LogicalTableModify; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.schema.Table; +import org.beam.sdk.java.sql.rel.BeamIOSinkRel; +import org.beam.sdk.java.sql.rel.BeamLogicalConvention; + +public class BeamIOSinkRule extends ConverterRule { + public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule(); + + private BeamIOSinkRule() { + super(LogicalTableModify.class, Convention.NONE, BeamLogicalConvention.INSTANCE, + "BeamIOSinkRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final TableModify tableModify = (TableModify) rel; + final RelNode input = tableModify.getInput(); + + final RelOptCluster cluster = tableModify.getCluster(); + final RelTraitSet traitSet = tableModify.getTraitSet().replace(BeamLogicalConvention.INSTANCE); + final RelOptTable relOptTable = tableModify.getTable(); + final Prepare.CatalogReader catalogReader = tableModify.getCatalogReader(); + final RelNode convertedInput = convert(input, + input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)); + final TableModify.Operation operation = tableModify.getOperation(); + final List updateColumnList = tableModify.getUpdateColumnList(); + final List sourceExpressionList = tableModify.getSourceExpressionList(); + final boolean flattened = tableModify.isFlattened(); + + final Table table = tableModify.getTable().unwrap(Table.class); + + switch (table.getJdbcTableType()) { + case STREAM: + if (operation != TableModify.Operation.INSERT) { + throw new UnsupportedOperationException( + String.format("Streams doesn't support %s modify operation", operation)); + } + return new BeamIOSinkRel(cluster, traitSet.replace(BeamLogicalConvention.INSTANCE), + relOptTable, catalogReader, convertedInput, operation, updateColumnList, + sourceExpressionList, flattened); + default: + throw new IllegalArgumentException( + String.format("Unsupported table type: %s", table.getJdbcTableType())); + } + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java new file mode 100644 index 000000000000..1465133dd5b6 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java @@ -0,0 +1,27 @@ +package org.beam.sdk.java.sql.rule; + +import org.apache.calcite.plan.Convention; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.logical.LogicalTableScan; +import org.beam.sdk.java.sql.rel.BeamIOSourceRel; +import org.beam.sdk.java.sql.rel.BeamLogicalConvention; + +public class BeamIOSourceRule extends ConverterRule { + public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule(); + + private BeamIOSourceRule() { + super(LogicalTableScan.class, Convention.NONE, BeamLogicalConvention.INSTANCE, + "BeamIOSourceRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final TableScan scan = (TableScan) rel; + + return new BeamIOSourceRel(scan.getCluster(), + scan.getTraitSet().replace(BeamLogicalConvention.INSTANCE), scan.getTable()); + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java new file mode 100644 index 000000000000..8daf8dacf74b --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java @@ -0,0 +1,46 @@ +/** + * 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.beam.sdk.java.sql.rule; + +import org.apache.calcite.plan.Convention; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.logical.LogicalProject; +import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +import org.beam.sdk.java.sql.rel.BeamProjectRel; + +public class BeamProjectRule extends ConverterRule { + public static final BeamProjectRule INSTANCE = new BeamProjectRule(); + + private BeamProjectRule() { + super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, + "BeamProjectRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final Project project = (Project) rel; + final RelNode input = project.getInput(); + + return new BeamProjectRel(project.getCluster(), + project.getTraitSet().replace(BeamLogicalConvention.INSTANCE), + convert(input, input.getTraitSet().replace(BeamLogicalConvention.INSTANCE)), + project.getProjects(), project.getRowType()); + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java new file mode 100644 index 000000000000..bfce4981fb6f --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java @@ -0,0 +1,8 @@ +/** + * + */ +/** + * @author mingmxu + * + */ +package org.beam.sdk.java.sql.rule; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java new file mode 100644 index 000000000000..67c3608db78b --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java @@ -0,0 +1,97 @@ +package org.beam.sdk.java.sql.schema; + +import java.io.Serializable; + +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.calcite.DataContext; +import org.apache.calcite.linq4j.Enumerable; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.schema.ScannableTable; +import org.apache.calcite.schema.Schema.TableType; +import org.apache.calcite.schema.Statistic; +import org.apache.calcite.schema.Statistics; + +/** + * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}. + */ +public abstract class BaseBeamTable implements ScannableTable, Serializable { + + /** + * + */ + private static final long serialVersionUID = -1262988061830914193L; + private RelProtoDataType protoRowType; + // A transform to convert from a rawRecord of input + private PTransform, PCollection> sourceConverter; + // A transform to convert one record to a rawRecord for output + private PTransform, PCollection> sinkConcerter; + + public BaseBeamTable(RelProtoDataType protoRowType, + PTransform, PCollection> sourceConverter, + PTransform, PCollection> sinkConcerter) { + this.protoRowType = protoRowType; + this.sourceConverter = sourceConverter; + this.sinkConcerter = sinkConcerter; + } + + /** + * In Beam SQL, there's no difference between a batch query and a streaming + * query. {@link BeamIOType} is used to validate the sources. + */ + public abstract BeamIOType getSourceType(); + + /** + * create a READ PTransform. + * + * @return + */ + public abstract PTransform> buildReadTransform(); + + /** + * create a WRITE PTransform + * + * @return + */ + public abstract PTransform, PDone> buildWriteTransform(); + + public PTransform, PCollection> getSourceConverter() { + return sourceConverter; + } + + public PTransform, PCollection> getSinkConcerter() { + return sinkConcerter; + } + + @Override + public Enumerable scan(DataContext root) { + // not used as Beam SQL uses its own execution engine + return null; + } + + @Override + public RelDataType getRowType(RelDataTypeFactory typeFactory) { + return protoRowType.apply(typeFactory); + } + + /** + * Not used {@link Statistic} to optimize the plan + */ + @Override + public Statistic getStatistic() { + return Statistics.UNKNOWN; + } + + /** + * all sources are treated as TABLE in Beam SQL. + */ + @Override + public TableType getJdbcTableType() { + return TableType.TABLE; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java new file mode 100644 index 000000000000..ad9bb84007f4 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java @@ -0,0 +1,11 @@ +package org.beam.sdk.java.sql.schema; + +import java.io.Serializable; + +/** + * Type as a source IO, determined whether it's a STREAMING process, or batch + * process. + */ +public enum BeamIOType implements Serializable { + BOUNDED, UNBOUNDED; +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java new file mode 100644 index 000000000000..85cd180f9e06 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java @@ -0,0 +1,51 @@ +package org.beam.sdk.java.sql.schema; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; + +@DefaultCoder(AvroCoder.class) +public class BeamSQLRecordType implements Serializable { + /** + * + */ + private static final long serialVersionUID = -5318734648766104712L; + private List fieldsName = new ArrayList<>(); + private List fieldsType = new ArrayList<>(); + + public static BeamSQLRecordType from(RelDataType tableInfo) { + BeamSQLRecordType record = new BeamSQLRecordType(); + for (RelDataTypeField f : tableInfo.getFieldList()) { + record.fieldsName.add(f.getName()); + record.fieldsType.add(f.getType().getSqlTypeName().getName()); + } + return record; + } + + public List getFieldsName() { + return fieldsName; + } + + public void setFieldsName(List fieldsName) { + this.fieldsName = fieldsName; + } + + public List getFieldsType() { + return fieldsType; + } + + public void setFieldsType(List fieldsType) { + this.fieldsType = fieldsType; + } + + @Override + public String toString() { + return "RecordType [fieldsName=" + fieldsName + ", fieldsType=" + fieldsType + "]"; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java new file mode 100644 index 000000000000..8c521e3130aa --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java @@ -0,0 +1,114 @@ +package org.beam.sdk.java.sql.schema; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.calcite.sql.type.SqlTypeName; + +@DefaultCoder(AvroCoder.class) +public class BeamSQLRow implements Serializable { + /** + * + */ + private static final long serialVersionUID = 4569220242480160895L; + + private Map dataMap = new HashMap<>(); + @Nullable + private BeamSQLRecordType dataType; + + @Deprecated + public BeamSQLRow() { + } + + public BeamSQLRow(BeamSQLRecordType dataType) { + super(); + this.dataType = dataType; + } + + public void addField(String fieldName, Object fieldValue) { + if (fieldValue != null) { + dataMap.put(fieldName, fieldValue.toString()); + } else { + // dataMap.put(fieldName, null); + } + } + + public Object getFieldValue(int fieldIdx) { + return getFieldValue(dataType.getFieldsName().get(fieldIdx), + dataType.getFieldsType().get(fieldIdx)); + } + + public Object getFieldValue(String fieldName) { + if (dataType.getFieldsName().indexOf(fieldName) == -1) { + return null; + } + return getFieldValue(fieldName, + dataType.getFieldsType().get(dataType.getFieldsName().indexOf(fieldName))); + } + + private Object getFieldValue(String fieldName, String fieldType) { + if (dataMap.get(fieldName) == null) { + return null; + } + switch (SqlTypeName.valueOf(fieldType)) { + case INTEGER: + return Integer.valueOf(dataMap.get(fieldName)); + case VARCHAR: + return dataMap.get(fieldName); + case TIMESTAMP: //TODO + case BIGINT: + return Long.valueOf(dataMap.get(fieldName)); + default: + return dataMap.get(fieldName); + } + } + + public Map getDataMap() { + return dataMap; + } + + public void setDataMap(HashMap dataMap) { + this.dataMap = dataMap; + } + + public BeamSQLRecordType getDataType() { + return dataType; + } + + public void setDataType(BeamSQLRecordType dataType) { + this.dataType = dataType; + } + + @Override + public String toString() { + return "RecordRow [dataMap=" + dataMap + ", dataType=" + dataType + "]"; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((dataMap == null) ? 0 : dataMap.hashCode()); + result = prime * result + ((dataType == null) ? 0 : dataType.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) + return true; + if (obj == null) + return false; + if (getClass() != obj.getClass()) + return false; + BeamSQLRow other = (BeamSQLRow) obj; + return toString().equals(other.toString()); + } + + +} + diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java new file mode 100644 index 000000000000..289f79d6ec8b --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -0,0 +1,98 @@ +package org.beam.sdk.java.sql.schema.kafka; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.util.List; +import java.util.Map; + +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.io.kafka.KafkaRecord; +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.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.BeamIOType; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +public class BeamKafkaTable extends BaseBeamTable> { + + /** + * + */ + private static final long serialVersionUID = -634715473399906527L; + + private String bootstrapServers; + private List topics; + private Map configUpdates; + + private BeamKafkaTable(RelProtoDataType protoRowType, + PTransform>, PCollection> sourceConverter, + PTransform, PCollection>> sinkConcerter) { + super(protoRowType, sourceConverter, sinkConcerter); + } + + public BeamKafkaTable(RelProtoDataType protoRowType, + PTransform>, PCollection> sourceConverter, + PTransform, PCollection>> sinkConcerter, + String bootstrapServers, List topics) { + super(protoRowType, sourceConverter, sinkConcerter); + this.bootstrapServers = bootstrapServers; + this.topics = topics; + } + + public BeamKafkaTable updateConsumerProperties(Map configUpdates) { + this.configUpdates = configUpdates; + return this; + } + + @Override + public BeamIOType getSourceType() { + return BeamIOType.UNBOUNDED; + } + + @Override + public PTransform>> buildReadTransform() { + return KafkaIO.read().withBootstrapServers(this.bootstrapServers) + .withTopics(this.topics).updateConsumerProperties(configUpdates) + .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()); + } + + @Override + public PTransform>, PDone> buildWriteTransform() { + checkArgument(topics != null && topics.size() == 0, + "Only one topic can be acceptable as output."); + + return new PTransform>, PDone>() { + /** + * + */ + private static final long serialVersionUID = 1136964183593770265L; + + @Override + public PDone expand(PCollection> input) { + return input.apply("toKafkaKV", + ParDo.of(new DoFn, KV>() { + /** + * + */ + private static final long serialVersionUID = 6265036192598208789L; + + @ProcessElement + public void processElement(ProcessContext ctx) { + ctx.output(ctx.element().getKV()); + } + })).apply("writeToKafka", + KafkaIO.write().withBootstrapServers(bootstrapServers) + .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) + .withValueCoder(ByteArrayCoder.of())); + } + }; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java new file mode 100644 index 000000000000..9edb03f2650e --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java @@ -0,0 +1,40 @@ +package org.beam.sdk.java.sql.schema.kafka; + +import org.apache.beam.sdk.io.kafka.KafkaRecord; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +public class KafkaCSVRecordTransform { + + + public static class ReaderTransform extends PTransform>, PCollection>{ + /** + * + */ + private static final long serialVersionUID = 7613394830984433222L; + + @Override + public PCollection expand(PCollection> input) { + // TODO Auto-generated method stub + return null; + } + + } + + public static class SinkerTransform extends PTransform, PCollection>>{ + + /** + * + */ + private static final long serialVersionUID = -722396312765710736L; + + @Override + public PCollection> expand(PCollection input) { + // TODO Auto-generated method stub + return null; + } + + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java new file mode 100644 index 000000000000..75cb2853451d --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java @@ -0,0 +1,8 @@ +/** + * + */ +/** + * table schema for KafkaIO + * + */ +package org.beam.sdk.java.sql.schema.kafka; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java new file mode 100644 index 000000000000..350ae3677885 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * define table schema, to map with Beam IO components + * + */ +package org.beam.sdk.java.sql.schema; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java new file mode 100644 index 000000000000..83cd77a422d4 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java @@ -0,0 +1,47 @@ +package org.beam.sdk.java.sql.transform; + +import org.apache.beam.sdk.transforms.DoFn; +import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.springframework.expression.Expression; +import org.springframework.expression.ExpressionParser; +import org.springframework.expression.spel.SpelParserConfiguration; +import org.springframework.expression.spel.standard.SpelExpressionParser; +import org.springframework.expression.spel.support.StandardEvaluationContext; + +public class BeamSQLFilterFn extends DoFn { + /** + * + */ + private static final long serialVersionUID = -1256111753670606705L; + + private String stepName; + private String filterInString; + private Expression expression; + + public BeamSQLFilterFn(String stepName, String filterInString) { + super(); + this.stepName = stepName; + this.filterInString = filterInString; + } + + @Setup + public void setup() { + SpelParserConfiguration config = new SpelParserConfiguration(true, true); + ExpressionParser parser = new SpelExpressionParser(config); + expression = parser.parseExpression(filterInString); + } + + @ProcessElement + public void processElement(ProcessContext c) { + BeamSQLRow map = c.element(); + + StandardEvaluationContext mapContext = new StandardEvaluationContext(); + mapContext.setVariable("map", map); + boolean trueValue = expression.getValue(mapContext, Boolean.class); + + if (trueValue) { + c.output(map); + } + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java new file mode 100644 index 000000000000..094973208d8f --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java @@ -0,0 +1,24 @@ +package org.beam.sdk.java.sql.transform; + +import org.apache.beam.sdk.transforms.DoFn; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +public class BeamSQLOutputToConsoleFn extends DoFn { + /** + * + */ + private static final long serialVersionUID = -1256111753670606705L; + + private String stepName; + + public BeamSQLOutputToConsoleFn(String stepName) { + super(); + this.stepName = stepName; + } + + @ProcessElement + public void processElement(ProcessContext c) { + System.out.println("Output: " + c.element().getDataMap()); + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java new file mode 100644 index 000000000000..96faa4e820b7 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java @@ -0,0 +1,76 @@ +package org.beam.sdk.java.sql.transform; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; +import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.springframework.expression.Expression; +import org.springframework.expression.ExpressionParser; +import org.springframework.expression.spel.SpelParserConfiguration; +import org.springframework.expression.spel.standard.SpelExpressionParser; +import org.springframework.expression.spel.support.StandardEvaluationContext; + +public class BeamSQLProjectFn extends DoFn { + + /** + * + */ + private static final long serialVersionUID = -1046605249999014608L; + private String stepName; + private BeamSQLRecordType recordType; + private List rules; + + private List ruleExps; + + public BeamSQLProjectFn(String stepName, BeamSQLRecordType recordType, List rules) { + super(); + this.stepName = stepName; + this.recordType = recordType; + this.rules = rules; + } + + @Setup + public void setup() { + ruleExps = new ArrayList<>(rules.size()); + SpelParserConfiguration config = new SpelParserConfiguration(true, true); + ExpressionParser parser = new SpelExpressionParser(config); + for (int idx = 0; idx < rules.size(); ++idx) { + if (rules.get(idx).getType().equals(ProjectType.RexCall) + || rules.get(idx).getType().equals(ProjectType.RexLiteral)) { + ruleExps.add(parser.parseExpression(rules.get(idx).getProjectExp())); + } else { + ruleExps.add(null); + } + } + } + + @ProcessElement + public void processElement(ProcessContext c) { + StandardEvaluationContext mapContext = new StandardEvaluationContext(); + mapContext.setVariable("map", c.element()); + + BeamSQLRow outRow = new BeamSQLRow(recordType); + for (int idx = 0; idx < rules.size(); ++idx) { + ProjectRule rule = rules.get(idx); + switch (rule.getType()) { + case RexInputRef: + outRow.addField(recordType.getFieldsName().get(idx), + c.element().getFieldValue(rule.getSourceIndex())); + break; + case RexCall: + case RexLiteral: + outRow.addField(recordType.getFieldsName().get(idx), + ruleExps.get(idx).getValue(mapContext)); + break; + default: + break; + } + } + + c.output(outRow); + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/CalciteToSpEL.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/CalciteToSpEL.java new file mode 100644 index 000000000000..86fd9b6b1812 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/CalciteToSpEL.java @@ -0,0 +1,65 @@ +package org.beam.sdk.java.sql.transform; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; + +import com.ebay.dss.tora.beam_sql_poc.planner.BeamSqlUnsupportedException; +import com.google.common.base.Joiner; + +public class CalciteToSpEL { + + public static String rexcall2SpEL(RexCall cdn, RecordType recordType) { + List parts = new ArrayList<>(); + for (RexNode subcdn : cdn.operands) { + if (subcdn instanceof RexCall) { + parts.add(rexcall2SpEL((RexCall) subcdn, recordType)); + } else { + parts.add(subcdn instanceof RexInputRef + ? "#map.getFieldValue('" + + recordType.getFieldsName().get(((RexInputRef) subcdn).getIndex()) + "')" + : subcdn.toString()); + } + } + + String opName = cdn.op.getName(); + switch (cdn.op.getClass().getSimpleName()) { + case "SqlMonotonicBinaryOperator": // +-* + case "SqlBinaryOperator": // > < = >= <= <> OR AND || / . + switch (cdn.op.getName().toUpperCase()) { + case "AND": + return String.format(" ( %s ) ", Joiner.on("&&").join(parts) ); + case "OR": + return String.format(" ( %s ) ", Joiner.on("||").join(parts) ); + case "=": + return String.format(" ( %s ) ", Joiner.on("==").join(parts) ); + case "<>": + return String.format(" ( %s ) ", Joiner.on("!=").join(parts) ); + default: + return String.format(" ( %s ) ", Joiner.on(cdn.op.getName().toUpperCase()).join(parts) ); + } + case "SqlCaseOperator": // CASE + return String.format(" (%s ? %s : %s)", parts.get(0), parts.get(1), parts.get(2)); + case "SqlCastFunction": // CAST + return parts.get(0); + case "SqlPostfixOperator": + switch (opName.toUpperCase()) { + case "IS NULL": + return String.format(" null == %s ", parts.get(0)); + case "IS NOT NULL": + return String.format(" null != %s ", parts.get(0)); + default: + throw new BeamSqlUnsupportedException(); + } + case "SqlFloorFunction": + return String.format("%s / %s", parts.get(0), 3600000); +// throw new BeamSqlUnsupportedException("TODO"); + default: + throw new BeamSqlUnsupportedException(); + } + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectRule.java new file mode 100644 index 000000000000..5f0ef5318ca7 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectRule.java @@ -0,0 +1,48 @@ +package org.beam.sdk.java.sql.transform; + +import java.io.Serializable; + +public class ProjectRule implements Serializable { + /** + * + */ + private static final long serialVersionUID = 6166324769404546121L; + private ProjectType type; + private int sourceIndex;// for RexInputRef + + private String projectExp; + + public ProjectRule() { + } + + public ProjectType getType() { + return type; + } + + public void setType(ProjectType type) { + this.type = type; + } + + public String getProjectExp() { + return projectExp; + } + + public void setProjectExp(String projectExp) { + this.projectExp = projectExp; + } + + public int getSourceIndex() { + return sourceIndex; + } + + public void setSourceIndex(int sourceIndex) { + this.sourceIndex = sourceIndex; + } + + @Override + public String toString() { + return "ProjectRule [type=" + type + ", sourceIndex=" + sourceIndex + ", projectExp=" + + projectExp + "]"; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectType.java new file mode 100644 index 000000000000..50488afd034d --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectType.java @@ -0,0 +1,7 @@ +package org.beam.sdk.java.sql.transform; + +import java.io.Serializable; + +public enum ProjectType implements Serializable { + RexLiteral, RexInputRef, RexCall; +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java new file mode 100644 index 000000000000..de6377db41d1 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java @@ -0,0 +1,8 @@ +/** + * + */ +/** + * @author mingmxu + * + */ +package org.beam.sdk.java.sql.transform; \ No newline at end of file From 673c25871904124010fe59eed0bcb0c9a9161100 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Thu, 9 Mar 2017 11:27:50 -0800 Subject: [PATCH 02/15] save work: BeamSQLSpELExecutor as an executor for relational operators; --- ...BeamSqlRunner.java => BeamSqlExample.java} | 18 ++- .../BeamSQLExpressionExecutor.java | 8 ++ .../sql/interpreter/BeamSQLSpELExecutor.java | 108 ++++++++++++++++++ .../CalciteToSpEL.java | 15 +-- .../ProjectRule.java | 2 +- .../ProjectType.java | 2 +- .../java/sql/planner/BeamQueryPlanner.java | 2 +- .../beam/sdk/java/sql/schema/BeamSQLRow.java | 4 + .../java/sql/schema/kafka/BeamKafkaTable.java | 2 +- .../java/sql/transform/BeamSQLFilterFn.java | 34 +++--- .../java/sql/transform/BeamSQLProjectFn.java | 57 +++------ 11 files changed, 172 insertions(+), 80 deletions(-) rename sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/{BeamSqlRunner.java => BeamSqlExample.java} (82%) create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java rename sdks/java/sql/src/main/java/org/beam/sdk/java/sql/{transform => interpreter}/CalciteToSpEL.java (77%) rename sdks/java/sql/src/main/java/org/beam/sdk/java/sql/{transform => interpreter}/ProjectRule.java (95%) rename sdks/java/sql/src/main/java/org/beam/sdk/java/sql/{transform => interpreter}/ProjectType.java (73%) diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlRunner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java similarity index 82% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlRunner.java rename to sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java index b17d6801e25c..9d283a9563c1 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlRunner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java @@ -22,10 +22,11 @@ import org.apache.calcite.tools.Frameworks; import org.apache.calcite.util.ImmutableBitSet; import org.beam.sdk.java.sql.planner.BeamQueryPlanner; +import org.beam.sdk.java.sql.schema.kafka.BeamKafkaTable; import com.google.common.collect.ImmutableList; -public class BeamSqlRunner { +public class BeamSqlExample { private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); private final SchemaPlus schema = Frameworks.createRootSchema(true); @@ -50,18 +51,13 @@ public void explainAndRun(String sqlString) throws IOException, SQLException { } public static void main(String[] args) throws IOException, SQLException { - BeamSqlRunner runner = new BeamSqlRunner(); + BeamSqlExample runner = new BeamSqlExample(); runner.initTables(); // case 2: insert into

() select STREAM from //
from - String sql = "SELECT " + " SITEID, PAGEID as new_pageId,floor(EVENTTIMESTAMP TO HOUR) AS EVENT_HOUR" - + ", COUNT(*) AS SIZE " + "FROM SOJ_EVENT " - + "WHERE SITEID >= 0 " + "GROUP BY SITEID, PAGEID, floor(EVENTTIMESTAMP TO HOUR)"; - - sql = "SELECT " + " SITEID, PAGEID, HOP_START(EVENTTIMESTAMP, INTERVAL '1' HOUR, INTERVAL '3' HOUR) " - + ", COUNT(*) AS SIZE " + "FROM SOJ_EVENT " - + "WHERE SITEID >= 0 " + "GROUP BY SITEID, PAGEID, HOP(EVENTTIMESTAMP, INTERVAL '1' HOUR, INTERVAL '3' HOUR) "; + String sql = "SELECT " + " SITEID, PAGEID as new_pageId " + "FROM SOJ_EVENT " + + "WHERE SITEID >= 0 "; runner.explainAndRun(sql); } @@ -97,7 +93,7 @@ public RelDataType apply(RelDataTypeFactory a0) { stat = Statistics.UNKNOWN; -// addTable(tableName, -// new BeamKafkaTable(protoRowType, stat, bootstrapServer, topic)); + addTable(tableName, + new BeamKafkaTable(protoRowType, null, null)); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java index fb57bfd02f99..cfbfa6acabf4 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java @@ -1,7 +1,15 @@ package org.beam.sdk.java.sql.interpreter; import java.io.Serializable; +import java.util.List; + +import org.beam.sdk.java.sql.schema.BeamSQLRow; public interface BeamSQLExpressionExecutor extends Serializable { + public void prepare(); + + public List execute(BeamSQLRow inputRecord); + + public void close(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java new file mode 100644 index 000000000000..4e633a5969f2 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java @@ -0,0 +1,108 @@ +package org.beam.sdk.java.sql.interpreter; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.beam.sdk.java.sql.planner.BeamSqlUnsupportedException; +import org.beam.sdk.java.sql.rel.BeamFilterRel; +import org.beam.sdk.java.sql.rel.BeamProjectRel; +import org.beam.sdk.java.sql.rel.BeamRelNode; +import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.springframework.expression.Expression; +import org.springframework.expression.ExpressionParser; +import org.springframework.expression.spel.SpelParserConfiguration; +import org.springframework.expression.spel.standard.SpelExpressionParser; +import org.springframework.expression.spel.support.StandardEvaluationContext; + +public class BeamSQLSpELExecutor implements BeamSQLExpressionExecutor { + /** + * + */ + private static final long serialVersionUID = 6777232573390074408L; + + private List spelString; + private List spelExpressions; + + public BeamSQLSpELExecutor(BeamRelNode relNode){ + this.spelString = new ArrayList<>(); + if(relNode instanceof BeamFilterRel){ + String filterSpEL = CalciteToSpEL.rexcall2SpEL( (RexCall) ((BeamFilterRel) relNode).getCondition() ); + spelString.add(filterSpEL); + }else if(relNode instanceof BeamProjectRel){ + List projectRules = createProjectRules((BeamProjectRel) relNode); + for (int idx = 0; idx < projectRules.size(); ++idx) { + if (projectRules.get(idx).getType().equals(ProjectType.RexCall) + || projectRules.get(idx).getType().equals(ProjectType.RexLiteral)) { + spelString.add(projectRules.get(idx).getProjectExp()); + } else { + spelString.add(null); //TODO + } + } + }else{ + throw new BeamSqlUnsupportedException(String.format("%s is not supported yet", relNode.getClass().toString())); + } + } + + @Override + public void prepare() { + this.spelExpressions = new ArrayList<>(); + + SpelParserConfiguration config = new SpelParserConfiguration(true, true); + ExpressionParser parser = new SpelExpressionParser(config); + for(String el : spelString){ + spelExpressions.add(parser.parseExpression(el)); + } + } + + @Override + public List execute(BeamSQLRow inputRecord) { + StandardEvaluationContext inContext = new StandardEvaluationContext(); + inContext.setVariable("in", inputRecord); + + List results = new ArrayList<>(); + for(Expression ep : spelExpressions){ + results.add(ep.getValue(inContext)); + } + return results; + } + + @Override + public void close() { + + + } + + private List createProjectRules(BeamProjectRel projectRel) { + List rules = new ArrayList<>(); + + List exps = projectRel.getProjects(); + + for (int idx = 0; idx < exps.size(); ++idx) { + RexNode node = exps.get(idx); + ProjectRule rule = new ProjectRule(); + + if (node instanceof RexLiteral) { + rule.setType(ProjectType.RexLiteral); + rule.setProjectExp(((RexLiteral) node).getValue() + ""); + }else{ + +// if (node instanceof RexInputRef) { +// rule.setType(ProjectType.RexInputRef); +// rule.setSourceIndex( ((RexInputRef) node).getIndex() ); +// } +// if (node instanceof RexCall) { +// rule.setType(ProjectType.RexCall); + + rule.setProjectExp(CalciteToSpEL.rexcall2SpEL((RexCall) node)); + } + rules.add(rule); + } + + return rules; + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/CalciteToSpEL.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java similarity index 77% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/CalciteToSpEL.java rename to sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java index 86fd9b6b1812..1f1511dd2c6c 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/CalciteToSpEL.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java @@ -1,4 +1,4 @@ -package org.beam.sdk.java.sql.transform; +package org.beam.sdk.java.sql.interpreter; import java.util.ArrayList; import java.util.List; @@ -6,21 +6,21 @@ import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; +import org.beam.sdk.java.sql.planner.BeamSqlUnsupportedException; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import com.ebay.dss.tora.beam_sql_poc.planner.BeamSqlUnsupportedException; import com.google.common.base.Joiner; public class CalciteToSpEL { - public static String rexcall2SpEL(RexCall cdn, RecordType recordType) { + public static String rexcall2SpEL(RexCall cdn) { List parts = new ArrayList<>(); for (RexNode subcdn : cdn.operands) { if (subcdn instanceof RexCall) { - parts.add(rexcall2SpEL((RexCall) subcdn, recordType)); + parts.add(rexcall2SpEL((RexCall) subcdn)); } else { parts.add(subcdn instanceof RexInputRef - ? "#map.getFieldValue('" - + recordType.getFieldsName().get(((RexInputRef) subcdn).getIndex()) + "')" + ? "#in.getFieldValue("+ ((RexInputRef) subcdn).getIndex() + ")" : subcdn.toString()); } } @@ -54,9 +54,6 @@ public static String rexcall2SpEL(RexCall cdn, RecordType recordType) { default: throw new BeamSqlUnsupportedException(); } - case "SqlFloorFunction": - return String.format("%s / %s", parts.get(0), 3600000); -// throw new BeamSqlUnsupportedException("TODO"); default: throw new BeamSqlUnsupportedException(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectRule.java rename to sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java index 5f0ef5318ca7..39b975a9aa54 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java @@ -1,4 +1,4 @@ -package org.beam.sdk.java.sql.transform; +package org.beam.sdk.java.sql.interpreter; import java.io.Serializable; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java similarity index 73% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectType.java rename to sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java index 50488afd034d..fcd1c51c89d0 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/ProjectType.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java @@ -1,4 +1,4 @@ -package org.beam.sdk.java.sql.transform; +package org.beam.sdk.java.sql.interpreter; import java.io.Serializable; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java index 7e83c40d5add..ca4724f06712 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java @@ -103,7 +103,7 @@ private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException System.out.println("SQLPlan>\n" + RelOptUtil.toString(relNode)); // PlannerImpl.transform() optimizes RelNode with ruleset - return planner.transform(1, traitSet.replace(BeamLogicalConvention.INSTANCE), relNode); + return planner.transform(0, traitSet.replace(BeamLogicalConvention.INSTANCE), relNode); } private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java index 8c521e3130aa..018ec9f6040e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java @@ -36,6 +36,10 @@ public void addField(String fieldName, Object fieldValue) { // dataMap.put(fieldName, null); } } + + public void addField(int index, Object fieldValue){ + addField(dataType.getFieldsName().get(index), fieldValue); + } public Object getFieldValue(int fieldIdx) { return getFieldValue(dataType.getFieldsName().get(fieldIdx), diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index 289f79d6ec8b..8ff9847be8ec 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -31,7 +31,7 @@ public class BeamKafkaTable extends BaseBeamTable> { private List topics; private Map configUpdates; - private BeamKafkaTable(RelProtoDataType protoRowType, + public BeamKafkaTable(RelProtoDataType protoRowType, PTransform>, PCollection> sourceConverter, PTransform, PCollection>> sinkConcerter) { super(protoRowType, sourceConverter, sinkConcerter); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java index 83cd77a422d4..b9e5cca4a7be 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java @@ -1,12 +1,10 @@ package org.beam.sdk.java.sql.transform; +import java.util.List; + import org.apache.beam.sdk.transforms.DoFn; +import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; import org.beam.sdk.java.sql.schema.BeamSQLRow; -import org.springframework.expression.Expression; -import org.springframework.expression.ExpressionParser; -import org.springframework.expression.spel.SpelParserConfiguration; -import org.springframework.expression.spel.standard.SpelExpressionParser; -import org.springframework.expression.spel.support.StandardEvaluationContext; public class BeamSQLFilterFn extends DoFn { /** @@ -15,33 +13,33 @@ public class BeamSQLFilterFn extends DoFn { private static final long serialVersionUID = -1256111753670606705L; private String stepName; - private String filterInString; - private Expression expression; + private BeamSQLExpressionExecutor executor; - public BeamSQLFilterFn(String stepName, String filterInString) { + public BeamSQLFilterFn(String stepName, BeamSQLExpressionExecutor executor) { super(); this.stepName = stepName; - this.filterInString = filterInString; + this.executor = executor; } @Setup public void setup() { - SpelParserConfiguration config = new SpelParserConfiguration(true, true); - ExpressionParser parser = new SpelExpressionParser(config); - expression = parser.parseExpression(filterInString); + executor.prepare(); } @ProcessElement public void processElement(ProcessContext c) { - BeamSQLRow map = c.element(); + BeamSQLRow in = c.element(); - StandardEvaluationContext mapContext = new StandardEvaluationContext(); - mapContext.setVariable("map", map); - boolean trueValue = expression.getValue(mapContext, Boolean.class); + List result = executor.execute(in); - if (trueValue) { - c.output(map); + if ((boolean)result.get(0)) { + c.output(in); } } + + @Teardown + public void close(){ + executor.close(); + } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java index 96faa4e820b7..4d3e0ee94ebf 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java @@ -5,6 +5,8 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.Teardown; +import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; import org.beam.sdk.java.sql.schema.BeamSQLRecordType; import org.beam.sdk.java.sql.schema.BeamSQLRow; import org.springframework.expression.Expression; @@ -20,57 +22,36 @@ public class BeamSQLProjectFn extends DoFn { */ private static final long serialVersionUID = -1046605249999014608L; private String stepName; - private BeamSQLRecordType recordType; - private List rules; + private BeamSQLExpressionExecutor executor; + private BeamSQLRecordType outputRecordType; - private List ruleExps; - - public BeamSQLProjectFn(String stepName, BeamSQLRecordType recordType, List rules) { + public BeamSQLProjectFn(String stepName, BeamSQLExpressionExecutor executor, BeamSQLRecordType outputRecordType) { super(); this.stepName = stepName; - this.recordType = recordType; - this.rules = rules; + this.executor = executor; + this.outputRecordType = outputRecordType; } @Setup public void setup() { - ruleExps = new ArrayList<>(rules.size()); - SpelParserConfiguration config = new SpelParserConfiguration(true, true); - ExpressionParser parser = new SpelExpressionParser(config); - for (int idx = 0; idx < rules.size(); ++idx) { - if (rules.get(idx).getType().equals(ProjectType.RexCall) - || rules.get(idx).getType().equals(ProjectType.RexLiteral)) { - ruleExps.add(parser.parseExpression(rules.get(idx).getProjectExp())); - } else { - ruleExps.add(null); - } - } + executor.prepare(); } @ProcessElement public void processElement(ProcessContext c) { - StandardEvaluationContext mapContext = new StandardEvaluationContext(); - mapContext.setVariable("map", c.element()); - - BeamSQLRow outRow = new BeamSQLRow(recordType); - for (int idx = 0; idx < rules.size(); ++idx) { - ProjectRule rule = rules.get(idx); - switch (rule.getType()) { - case RexInputRef: - outRow.addField(recordType.getFieldsName().get(idx), - c.element().getFieldValue(rule.getSourceIndex())); - break; - case RexCall: - case RexLiteral: - outRow.addField(recordType.getFieldsName().get(idx), - ruleExps.get(idx).getValue(mapContext)); - break; - default: - break; - } + List results = executor.execute(c.element()); + + BeamSQLRow outRow = new BeamSQLRow(outputRecordType); + for(int idx=0; idx Date: Thu, 9 Mar 2017 11:49:57 -0800 Subject: [PATCH 03/15] save work, update buildBeamPipeline() --- .../sdk/java/sql/planner/BeamSQLRelUtils.java | 54 +++++++++++++++++++ .../beam/sdk/java/sql/rel/BeamFilterRel.java | 23 +++++++- .../beam/sdk/java/sql/rel/BeamProjectRel.java | 22 +++++++- 3 files changed, 96 insertions(+), 3 deletions(-) create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java new file mode 100644 index 000000000000..6de9ecb81f9e --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java @@ -0,0 +1,54 @@ +package org.beam.sdk.java.sql.planner; + +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.plan.volcano.RelSubset; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.sql.SqlExplainLevel; +import org.beam.sdk.java.sql.rel.BeamRelNode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BeamSQLRelUtils { + private static final Logger LOG = LoggerFactory.getLogger(BeamSQLRelUtils.class); + + private static final AtomicInteger sequence = new AtomicInteger(0); + private static final AtomicInteger classSequence = new AtomicInteger(0); + + public static String getStageName(BeamRelNode relNode) { + return relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + "_" + + sequence.getAndIncrement(); + } + + public static String getClassName(BeamRelNode relNode) { + return "Generated_" + relNode.getClass().getSimpleName().toUpperCase() + "_" + relNode.getId() + + "_" + classSequence.getAndIncrement(); + } + + public static BeamRelNode getBeamRelInput(RelNode input) { + if (input instanceof RelSubset) { + // go with known best input + input = ((RelSubset) input).getBest(); + } + return (BeamRelNode) input; + } + + public static String explain(final RelNode rel) { + return explain(rel, SqlExplainLevel.EXPPLAN_ATTRIBUTES); + } + + public static String explain(final RelNode rel, SqlExplainLevel detailLevel) { + String explain = ""; + try { + explain = RelOptUtil.toString(rel); + } catch (StackOverflowError e) { + LOG.error( + "StackOverflowError occurred while extracting plan. Please report it to the dev@ mailing list."); + LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e); + LOG.error( + "Forcing plan to empty string and continue... SQL Runner may not working properly after."); + } + return explain; + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java index ead31e99f66a..2e7068cb17c6 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java @@ -11,7 +11,12 @@ import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; +import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.sdk.java.sql.interpreter.BeamSQLSpELExecutor; import org.beam.sdk.java.sql.planner.BeamPipelineCreator; +import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; +import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.sdk.java.sql.transform.BeamSQLFilterFn; public class BeamFilterRel extends Filter implements BeamRelNode { @@ -27,8 +32,22 @@ public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) { @Override public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { - // TODO Auto-generated method stub - + + RelNode input = getInput(); + BeamSQLRelUtils.getBeamRelInput(input).buildBeamPipeline(planCreator); + + String stageName = BeamSQLRelUtils.getStageName(this); + + PCollection upstream = planCreator.getLatestStream(); + + BeamSQLExpressionExecutor executor = new BeamSQLSpELExecutor(this); + + PCollection projectStream = upstream.apply(stageName, + ParDo.of(new BeamSQLFilterFn(getRelTypeName(), executor))); + + planCreator.setLatestStream(projectStream); + + System.out.println("Build: apply_filter " + stageName ); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java index 00ede43dfacf..b3c586188bcc 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java @@ -13,7 +13,13 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.sdk.java.sql.interpreter.BeamSQLSpELExecutor; import org.beam.sdk.java.sql.planner.BeamPipelineCreator; +import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; +import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.sdk.java.sql.transform.BeamSQLProjectFn; public class BeamProjectRel extends Project implements BeamRelNode { @@ -39,7 +45,21 @@ public Project copy(RelTraitSet traitSet, RelNode input, List projects, @Override public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { - // TODO Auto-generated method stub + RelNode input = getInput(); + BeamSQLRelUtils.getBeamRelInput(input).buildBeamPipeline(planCreator); + + String stageName = BeamSQLRelUtils.getStageName(this); + + PCollection upstream = planCreator.getLatestStream(); + + BeamSQLExpressionExecutor executor = new BeamSQLSpELExecutor(this); + + PCollection projectStream = upstream.apply(stageName, + ParDo.of(new BeamSQLProjectFn(getRelTypeName(), executor, BeamSQLRecordType.from(rowType)))); + + planCreator.setLatestStream(projectStream); + + System.out.println("Build: apply_project " + stageName); } From 64f8e89e169151fd323cf1bb45432281a19cedef Mon Sep 17 00:00:00 2001 From: mingmxu Date: Thu, 9 Mar 2017 14:30:10 -0800 Subject: [PATCH 04/15] complete first query: SELECT FROM
WHERE --- sdks/java/sql/pom.xml | 41 ++++++++++ .../sdk/java/sql/examples/BeamSqlExample.java | 26 ++++-- .../sql/examples/RheosSourceTransform.java | 81 +++++++++++++++++++ .../sql/interpreter/BeamSQLSpELExecutor.java | 52 ++++++------ .../java/sql/interpreter/CalciteToSpEL.java | 1 - .../java/sql/planner/BeamPipelineCreator.java | 2 - .../sdk/java/sql/planner/BeamRuleSets.java | 13 --- .../beam/sdk/java/sql/rel/BeamFilterRel.java | 4 - .../beam/sdk/java/sql/rel/BeamIOSinkRel.java | 5 -- .../sdk/java/sql/rel/BeamIOSourceRel.java | 29 ++++--- .../beam/sdk/java/sql/rel/BeamProjectRel.java | 2 +- .../java/sql/schema/kafka/BeamKafkaTable.java | 2 +- .../java/sql/transform/BeamSQLFilterFn.java | 2 +- .../java/sql/transform/BeamSQLProjectFn.java | 8 -- 14 files changed, 190 insertions(+), 78 deletions(-) create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java diff --git a/sdks/java/sql/pom.xml b/sdks/java/sql/pom.xml index 9c29a450f6d2..9dff6ab3e4d4 100644 --- a/sdks/java/sql/pom.xml +++ b/sdks/java/sql/pom.xml @@ -20,6 +20,35 @@ 1.11.0 + + + + apache-snapshot + https://repository.apache.org/content/repositories/snapshots/ + + + apache-release + https://repository.apache.org/content/repositories/releases/ + + + ebaycentral-release + ebaycentral-release repository + http://ebaycentral/content/repositories/releases/ + + + raptor-snapshots + http://ebaycentral/content/repositories/snapshots/ + + + central + http://ebaycentral/content/repositories/central + + + raptor-thirdparty + http://ebaycentral/content/repositories/thirdparty + + + @@ -126,6 +155,12 @@ org.apache.beam beam-sdks-java-io-kafka + + + org.apache.kafka + kafka-clients + + org.springframework @@ -135,5 +170,11 @@ com.google.guava guava + + io.ebay.rheos + rheos-client + 0.0.4 + + diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java index 9d283a9563c1..a001bb1684fe 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java @@ -1,7 +1,11 @@ package org.beam.sdk.java.sql.examples; import java.io.IOException; +import java.io.Serializable; import java.sql.SQLException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.jdbc.JavaTypeFactoryImpl; @@ -21,12 +25,14 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.Frameworks; import org.apache.calcite.util.ImmutableBitSet; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.beam.sdk.java.sql.planner.BeamQueryPlanner; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; import org.beam.sdk.java.sql.schema.kafka.BeamKafkaTable; import com.google.common.collect.ImmutableList; -public class BeamSqlExample { +public class BeamSqlExample implements Serializable{ private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); private final SchemaPlus schema = Frameworks.createRootSchema(true); @@ -57,7 +63,7 @@ public static void main(String[] args) throws IOException, SQLException { // case 2: insert into
() select STREAM from //
from String sql = "SELECT " + " SITEID, PAGEID as new_pageId " + "FROM SOJ_EVENT " - + "WHERE SITEID >= 0 "; + + "WHERE SITEID > 0 "; runner.explainAndRun(sql); } @@ -67,18 +73,15 @@ void initTables() { "rheos-kafka-proxy-1.lvs02.dev.ebayc3.com:9092,rheos-kafka-proxy-2.lvs02.dev.ebayc3.com:9092", "behavior.pulsar.sojevent.total"); addTable("subrowverevent", "flink2-8332.lvs01.dev.ebayc3.com:9092", "subrowverevent"); - // addTable("externalorder", "flink2-8332.lvs01.dev.ebayc3.com:9092", - // "externalorder"); } void addTable(String tableName, String bootstrapServer, String topic) { - RelProtoDataType protoRowType = new RelProtoDataType() { + final RelProtoDataType protoRowType = new RelProtoDataType() { @Override public RelDataType apply(RelDataTypeFactory a0) { return a0.builder().add("EVENTTIMESTAMP", SqlTypeName.TIMESTAMP) .add("ITEMID", SqlTypeName.BIGINT).add("SITEID", SqlTypeName.INTEGER) .add("PAGEID", SqlTypeName.INTEGER).add("PAGENAME", SqlTypeName.VARCHAR) - // .add("APPLICATIONPAYLOAD", SqlTypeName.INTEGER) .build(); } }; @@ -93,7 +96,16 @@ public RelDataType apply(RelDataTypeFactory a0) { stat = Statistics.UNKNOWN; + Map consumerPara = new HashMap(); + consumerPara.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); // latest + // or + // earliest + consumerPara.put(ConsumerConfig.GROUP_ID_CONFIG, "toraframeworktest_sojeventproxy"); + consumerPara.put(ConsumerConfig.CLIENT_ID_CONFIG, "107b877c-2694-4298-beed-9007f54602aa"); + addTable(tableName, - new BeamKafkaTable(protoRowType, null, null)); + new BeamKafkaTable(protoRowType, new RheosSourceTransform(BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), null, bootstrapServer, Arrays.asList(topic)) + .updateConsumerProperties(consumerPara) + ); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java new file mode 100644 index 000000000000..f1daa504ce99 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java @@ -0,0 +1,81 @@ +package org.beam.sdk.java.sql.examples; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.avro.Schema.Field; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.beam.sdk.io.kafka.KafkaRecord; +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.values.PCollection; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +import io.ebay.rheos.kafka.client.StreamConnectorConfig; +import io.ebay.rheos.schema.avro.RheosEventDeserializer; +import io.ebay.rheos.schema.avro.SchemaRegistryAwareAvroDeserializerHelper; +import io.ebay.rheos.schema.event.RheosEvent; + +public class RheosSourceTransform extends PTransform>, PCollection>{ + private BeamSQLRecordType recordType; + + + public RheosSourceTransform(BeamSQLRecordType recordType) { + super(); + this.recordType = recordType; + } + + + @Override + public PCollection expand(PCollection> input) { + return input.apply("soureDecode", ParDo.of(new DoFn, BeamSQLRow>(){ + transient RheosEventDeserializer rheosDeserializer; + transient SchemaRegistryAwareAvroDeserializerHelper deserializerHelper; + + @Setup + public void setup() { + this.rheosDeserializer = new RheosEventDeserializer(); + + Map config = new HashMap<>(); + config.put(StreamConnectorConfig.RHEOS_SERVICES_URLS, "https://rheos-services.qa.ebay.com"); + + deserializerHelper = new SchemaRegistryAwareAvroDeserializerHelper<>(config, + GenericRecord.class); + } + + @ProcessElement + public void processElement(ProcessContext c) throws IOException { + byte[] rawBytes = c.element().getKV().getValue(); + RheosEvent event = rheosDeserializer.deserialize("", rawBytes); + + DatumReader reader = new GenericDatumReader( + deserializerHelper.getSchema(event.getSchemaId())); + Decoder decoder = DecoderFactory.get().binaryDecoder(event.toBytes(), null); + + GenericRecord record = reader.read(null, decoder); + Map k2k = new HashMap<>(); + for (Field f : record.getSchema().getFields()) { + k2k.put(f.name().toUpperCase(), f.name()); + } + +// BeamSQLRecordType recordType = BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl())); + BeamSQLRow values = new BeamSQLRow(recordType); + for (int idx = 0; idx < recordType.getFieldsName().size(); ++idx) { + values.addField(recordType.getFieldsName().get(idx), + record.get(k2k.get(recordType.getFieldsName().get(idx)))); + } + + c.output(values); + } + + })); + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java index 4e633a5969f2..4e27bedc3918 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java @@ -26,34 +26,36 @@ public class BeamSQLSpELExecutor implements BeamSQLExpressionExecutor { private List spelString; private List spelExpressions; - - public BeamSQLSpELExecutor(BeamRelNode relNode){ + + public BeamSQLSpELExecutor(BeamRelNode relNode) { this.spelString = new ArrayList<>(); - if(relNode instanceof BeamFilterRel){ - String filterSpEL = CalciteToSpEL.rexcall2SpEL( (RexCall) ((BeamFilterRel) relNode).getCondition() ); + if (relNode instanceof BeamFilterRel) { + String filterSpEL = CalciteToSpEL + .rexcall2SpEL((RexCall) ((BeamFilterRel) relNode).getCondition()); spelString.add(filterSpEL); - }else if(relNode instanceof BeamProjectRel){ + } else if (relNode instanceof BeamProjectRel) { List projectRules = createProjectRules((BeamProjectRel) relNode); for (int idx = 0; idx < projectRules.size(); ++idx) { - if (projectRules.get(idx).getType().equals(ProjectType.RexCall) - || projectRules.get(idx).getType().equals(ProjectType.RexLiteral)) { +// if (projectRules.get(idx).getType().equals(ProjectType.RexCall) +// || projectRules.get(idx).getType().equals(ProjectType.RexLiteral)) { spelString.add(projectRules.get(idx).getProjectExp()); - } else { - spelString.add(null); //TODO - } +// } else { +// spelString.add(null); // TODO +// } } - }else{ - throw new BeamSqlUnsupportedException(String.format("%s is not supported yet", relNode.getClass().toString())); + } else { + throw new BeamSqlUnsupportedException( + String.format("%s is not supported yet", relNode.getClass().toString())); } } @Override public void prepare() { this.spelExpressions = new ArrayList<>(); - + SpelParserConfiguration config = new SpelParserConfiguration(true, true); ExpressionParser parser = new SpelExpressionParser(config); - for(String el : spelString){ + for (String el : spelString) { spelExpressions.add(parser.parseExpression(el)); } } @@ -62,9 +64,9 @@ public void prepare() { public List execute(BeamSQLRow inputRecord) { StandardEvaluationContext inContext = new StandardEvaluationContext(); inContext.setVariable("in", inputRecord); - + List results = new ArrayList<>(); - for(Expression ep : spelExpressions){ + for (Expression ep : spelExpressions) { results.add(ep.getValue(inContext)); } return results; @@ -72,10 +74,9 @@ public List execute(BeamSQLRow inputRecord) { @Override public void close() { - - + } - + private List createProjectRules(BeamProjectRel projectRel) { List rules = new ArrayList<>(); @@ -90,17 +91,18 @@ private List createProjectRules(BeamProjectRel projectRel) { rule.setProjectExp(((RexLiteral) node).getValue() + ""); }else{ -// if (node instanceof RexInputRef) { -// rule.setType(ProjectType.RexInputRef); + if (node instanceof RexInputRef) { + rule.setType(ProjectType.RexInputRef); // rule.setSourceIndex( ((RexInputRef) node).getIndex() ); -// } -// if (node instanceof RexCall) { -// rule.setType(ProjectType.RexCall); - + rule.setProjectExp("#in.getFieldValue("+((RexInputRef) node).getIndex()+")"); + } + if (node instanceof RexCall) { + rule.setType(ProjectType.RexCall); rule.setProjectExp(CalciteToSpEL.rexcall2SpEL((RexCall) node)); } rules.add(rule); } + } return rules; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java index 1f1511dd2c6c..4514a7e9cd4f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java @@ -7,7 +7,6 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.beam.sdk.java.sql.planner.BeamSqlUnsupportedException; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; import com.google.common.base.Joiner; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java index 1273b4e7be29..c5ba926f7767 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java @@ -3,7 +3,6 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.ParDo; @@ -12,7 +11,6 @@ import org.beam.sdk.java.sql.schema.BaseBeamTable; import org.beam.sdk.java.sql.schema.BeamSQLRow; import org.beam.sdk.java.sql.transform.BeamSQLOutputToConsoleFn; -import org.joda.time.Duration; /** * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam pipeline. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java index 646c6ee033c8..a4b1f6b3bd6c 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java @@ -4,19 +4,6 @@ import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.rules.CalcMergeRule; -import org.apache.calcite.rel.rules.FilterCalcMergeRule; -import org.apache.calcite.rel.rules.FilterProjectTransposeRule; -import org.apache.calcite.rel.rules.FilterToCalcRule; -import org.apache.calcite.rel.rules.ProjectCalcMergeRule; -import org.apache.calcite.rel.rules.ProjectFilterTransposeRule; -import org.apache.calcite.rel.rules.ProjectRemoveRule; -import org.apache.calcite.rel.rules.ProjectToCalcRule; -import org.apache.calcite.rel.rules.PruneEmptyRules; -import org.apache.calcite.rel.rules.ReduceExpressionsRule; -import org.apache.calcite.rel.rules.SortRemoveRule; -import org.apache.calcite.rel.rules.UnionEliminatorRule; -import org.apache.calcite.rel.stream.StreamRules; import org.apache.calcite.tools.RuleSet; import org.beam.sdk.java.sql.rel.BeamRelNode; import org.beam.sdk.java.sql.rule.BeamFilterRule; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java index 2e7068cb17c6..fe0ff8ce034a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java @@ -1,15 +1,11 @@ package org.beam.sdk.java.sql.rel; -import java.util.ArrayList; -import java.util.List; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Filter; -import org.apache.calcite.rex.RexCall; -import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; import org.beam.sdk.java.sql.interpreter.BeamSQLSpELExecutor; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java index 3b0508cfb7d2..03944e4d8c9c 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java @@ -2,11 +2,6 @@ import java.util.List; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.kafka.KafkaIO; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitSet; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java index 8d78acc41d73..81052b870827 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java @@ -1,19 +1,16 @@ package org.beam.sdk.java.sql.rel; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.io.kafka.KafkaIO; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.core.TableScan; -import org.apache.kafka.clients.consumer.ConsumerConfig; import org.beam.sdk.java.sql.planner.BeamPipelineCreator; +import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +import com.google.common.base.Joiner; public class BeamIOSourceRel extends TableScan implements BeamRelNode { @@ -23,8 +20,20 @@ public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable @Override public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { - // TODO Auto-generated method stub - + + String sourceName = Joiner.on('.').join(getTable().getQualifiedName()).replace(".(STREAM)", ""); + + BaseBeamTable sourceTable = planCreator.getKafkaTables().get(sourceName); + + String stageName = BeamSQLRelUtils.getStageName(this); + + PCollection sourceStream = planCreator.getPipeline() + .apply(stageName, sourceTable.buildReadTransform()); + PCollection reformattedSourceStream = sourceStream.apply("sourceReformat", sourceTable.getSourceConverter()); + + planCreator.setLatestStream(reformattedSourceStream); + + System.out.println("Build: add_source " + sourceName); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java index b3c586188bcc..5f4d1f29c062 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java @@ -1,7 +1,7 @@ package org.beam.sdk.java.sql.rel; -import java.util.ArrayList; import java.util.List; + import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index 8ff9847be8ec..942705cbf165 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -31,7 +31,7 @@ public class BeamKafkaTable extends BaseBeamTable> { private List topics; private Map configUpdates; - public BeamKafkaTable(RelProtoDataType protoRowType, + protected BeamKafkaTable(RelProtoDataType protoRowType, PTransform>, PCollection> sourceConverter, PTransform, PCollection>> sinkConcerter) { super(protoRowType, sourceConverter, sinkConcerter); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java index b9e5cca4a7be..7158621fdace 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java @@ -32,7 +32,7 @@ public void processElement(ProcessContext c) { List result = executor.execute(in); - if ((boolean)result.get(0)) { + if ((Boolean)result.get(0)) { c.output(in); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java index 4d3e0ee94ebf..07e3faa4c038 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java @@ -1,19 +1,11 @@ package org.beam.sdk.java.sql.transform; -import java.util.ArrayList; import java.util.List; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.Teardown; import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; import org.beam.sdk.java.sql.schema.BeamSQLRecordType; import org.beam.sdk.java.sql.schema.BeamSQLRow; -import org.springframework.expression.Expression; -import org.springframework.expression.ExpressionParser; -import org.springframework.expression.spel.SpelParserConfiguration; -import org.springframework.expression.spel.standard.SpelExpressionParser; -import org.springframework.expression.spel.support.StandardEvaluationContext; public class BeamSQLProjectFn extends DoFn { From 616484c58a265fee31c64d66360689cd3ba71355 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Fri, 10 Mar 2017 13:48:49 -0800 Subject: [PATCH 05/15] save work: support INSERT --- .../sdk/java/sql/examples/BeamSqlExample.java | 35 ++++++++-------- .../java/sql/examples/RheosSinkTransform.java | 40 +++++++++++++++++++ .../sql/examples/RheosSourceTransform.java | 13 ++++-- .../java/sql/planner/BeamPipelineCreator.java | 8 +++- .../beam/sdk/java/sql/rel/BeamIOSinkRel.java | 30 +++++++++++++- .../sdk/java/sql/rel/BeamIOSourceRel.java | 2 +- .../sdk/java/sql/rule/BeamIOSinkRule.java | 1 + .../java/sql/schema/kafka/BeamKafkaTable.java | 39 ++++++++---------- .../sql/schema/kafka/KafkaKVExtractor.java | 36 +++++++++++++++++ 9 files changed, 156 insertions(+), 48 deletions(-) create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java index a001bb1684fe..43018459d04f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java @@ -32,7 +32,7 @@ import com.google.common.collect.ImmutableList; -public class BeamSqlExample implements Serializable{ +public class BeamSqlExample implements Serializable { private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); private final SchemaPlus schema = Frameworks.createRootSchema(true); @@ -59,12 +59,12 @@ public void explainAndRun(String sqlString) throws IOException, SQLException { public static void main(String[] args) throws IOException, SQLException { BeamSqlExample runner = new BeamSqlExample(); runner.initTables(); - + // case 2: insert into
() select STREAM from //
from - String sql = "SELECT " + " SITEID, PAGEID as new_pageId " + "FROM SOJ_EVENT " - + "WHERE SITEID > 0 "; - + String sql = "INSERT INTO subrowverevent(SITEID, PAGEID) " + "SELECT " + + " SITEID, PAGEID as new_pageId " + "FROM SOJ_EVENT " + "WHERE SITEID > 0 "; + runner.explainAndRun(sql); } @@ -81,31 +81,32 @@ void addTable(String tableName, String bootstrapServer, String topic) { public RelDataType apply(RelDataTypeFactory a0) { return a0.builder().add("EVENTTIMESTAMP", SqlTypeName.TIMESTAMP) .add("ITEMID", SqlTypeName.BIGINT).add("SITEID", SqlTypeName.INTEGER) - .add("PAGEID", SqlTypeName.INTEGER).add("PAGENAME", SqlTypeName.VARCHAR) - .build(); + .add("PAGEID", SqlTypeName.INTEGER).add("PAGENAME", SqlTypeName.VARCHAR).build(); } }; Direction dir = Direction.ASCENDING; RelFieldCollation collation = new RelFieldCollation(0, dir, NullDirection.UNSPECIFIED); Statistic stat = Statistics.of(5, ImmutableList.of(ImmutableBitSet.of(0)), ImmutableList.of(RelCollations.of(collation))); - - stat = Statistics.of(100d, - ImmutableList.of(), -RelCollations.createSingleton(0)); - + + stat = Statistics.of(100d, ImmutableList.of(), + RelCollations.createSingleton(0)); + stat = Statistics.UNKNOWN; - + Map consumerPara = new HashMap(); consumerPara.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); // latest // or // earliest consumerPara.put(ConsumerConfig.GROUP_ID_CONFIG, "toraframeworktest_sojeventproxy"); consumerPara.put(ConsumerConfig.CLIENT_ID_CONFIG, "107b877c-2694-4298-beed-9007f54602aa"); - + addTable(tableName, - new BeamKafkaTable(protoRowType, new RheosSourceTransform(BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), null, bootstrapServer, Arrays.asList(topic)) - .updateConsumerProperties(consumerPara) - ); + new BeamKafkaTable(protoRowType, + new RheosSourceTransform( + BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), + new RheosSinkTransform( + BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), + bootstrapServer, Arrays.asList(topic)).updateConsumerProperties(consumerPara)); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java new file mode 100644 index 000000000000..c5c344895def --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java @@ -0,0 +1,40 @@ +package org.beam.sdk.java.sql.examples; + +import java.io.IOException; + +import org.apache.beam.sdk.io.kafka.KafkaRecord; +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.DoFn.ProcessContext; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +public class RheosSinkTransform extends PTransform, PCollection>>{ + /** + * + */ + private static final long serialVersionUID = 6589679229923907701L; + private BeamSQLRecordType recordType; + + + public RheosSinkTransform(BeamSQLRecordType recordType) { + super(); + this.recordType = recordType; + } + + + @Override + public PCollection> expand(PCollection input) { + return input.apply("toKafkaRecord", ParDo.of(new DoFn>(){ + @ProcessElement + public void processElement(ProcessContext c) throws IOException { + c.output(KV.of(new byte[]{}, c.element().getDataMap().toString().getBytes())); + } + })); + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java index f1daa504ce99..59e72ee7ef74 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java @@ -14,6 +14,7 @@ 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.values.KV; import org.apache.beam.sdk.values.PCollection; import org.beam.sdk.java.sql.schema.BeamSQLRecordType; import org.beam.sdk.java.sql.schema.BeamSQLRow; @@ -23,7 +24,11 @@ import io.ebay.rheos.schema.avro.SchemaRegistryAwareAvroDeserializerHelper; import io.ebay.rheos.schema.event.RheosEvent; -public class RheosSourceTransform extends PTransform>, PCollection>{ +public class RheosSourceTransform extends PTransform>, PCollection>{ + /** + * + */ + private static final long serialVersionUID = -7803885128685230359L; private BeamSQLRecordType recordType; @@ -34,8 +39,8 @@ public RheosSourceTransform(BeamSQLRecordType recordType) { @Override - public PCollection expand(PCollection> input) { - return input.apply("soureDecode", ParDo.of(new DoFn, BeamSQLRow>(){ + public PCollection expand(PCollection> input) { + return input.apply("soureDecode", ParDo.of(new DoFn, BeamSQLRow>(){ transient RheosEventDeserializer rheosDeserializer; transient SchemaRegistryAwareAvroDeserializerHelper deserializerHelper; @@ -52,7 +57,7 @@ public void setup() { @ProcessElement public void processElement(ProcessContext c) throws IOException { - byte[] rawBytes = c.element().getKV().getValue(); + byte[] rawBytes = c.element().getValue(); RheosEvent event = rheosDeserializer.deserialize("", rawBytes); DatumReader reader = new GenericDatumReader( diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java index c5ba926f7767..cefdec5ab154 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java @@ -3,6 +3,10 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.io.kafka.KafkaRecord; +import org.apache.beam.sdk.io.kafka.KafkaRecordCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.ParDo; @@ -12,6 +16,8 @@ import org.beam.sdk.java.sql.schema.BeamSQLRow; import org.beam.sdk.java.sql.transform.BeamSQLOutputToConsoleFn; +import io.ebay.rheos.schema.event.RheosEvent; + /** * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam pipeline. * @@ -52,7 +58,7 @@ public void setLatestStream(PCollection latestStream) { this.latestStream = latestStream; } - public Map getKafkaTables() { + public Map getSourceTables() { return sourceTables; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java index 03944e4d8c9c..cc6372bd2c59 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java @@ -2,6 +2,11 @@ import java.util.List; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitSet; @@ -10,6 +15,11 @@ import org.apache.calcite.rel.core.TableModify; import org.apache.calcite.rex.RexNode; import org.beam.sdk.java.sql.planner.BeamPipelineCreator; +import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +import com.google.common.base.Joiner; public class BeamIOSinkRel extends TableModify implements BeamRelNode { public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, @@ -28,8 +38,24 @@ public RelNode copy(RelTraitSet traitSet, List inputs) { @Override public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { - // TODO Auto-generated method stub - + + RelNode input = getInput(); + BeamSQLRelUtils.getBeamRelInput(input).buildBeamPipeline(planCreator); + + String stageName = BeamSQLRelUtils.getStageName(this); + + PCollection upstream = planCreator.getLatestStream(); + + String sourceName = Joiner.on('.').join(getTable().getQualifiedName()); + + BaseBeamTable targetTable = planCreator.getSourceTables().get(sourceName); + + PCollection formattedOutput = upstream.apply("preformat_to_target", targetTable.getSinkConcerter()); + formattedOutput.apply("persistent", targetTable.buildWriteTransform()); + + planCreator.setHasPersistent(true); + + System.out.println("Build: add_persistent " + stageName); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java index 81052b870827..0fd4e13e7746 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java @@ -23,7 +23,7 @@ public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception String sourceName = Joiner.on('.').join(getTable().getQualifiedName()).replace(".(STREAM)", ""); - BaseBeamTable sourceTable = planCreator.getKafkaTables().get(sourceName); + BaseBeamTable sourceTable = planCreator.getSourceTables().get(sourceName); String stageName = BeamSQLRelUtils.getStageName(this); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java index 370191ae045a..bc75d4bd855c 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java @@ -43,6 +43,7 @@ public RelNode convert(RelNode rel) { final Table table = tableModify.getTable().unwrap(Table.class); switch (table.getJdbcTableType()) { + case TABLE: case STREAM: if (operation != TableModify.Operation.INSERT) { throw new UnsupportedOperationException( diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index 942705cbf165..773e7f3d375e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -2,6 +2,7 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.io.Serializable; import java.util.List; import java.util.Map; @@ -16,11 +17,13 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.calcite.rel.type.RelProtoDataType; +import org.beam.sdk.java.sql.examples.RheosSinkTransform; +import org.beam.sdk.java.sql.examples.RheosSourceTransform; import org.beam.sdk.java.sql.schema.BaseBeamTable; import org.beam.sdk.java.sql.schema.BeamIOType; import org.beam.sdk.java.sql.schema.BeamSQLRow; -public class BeamKafkaTable extends BaseBeamTable> { +public class BeamKafkaTable extends BaseBeamTable> implements Serializable { /** * @@ -32,14 +35,14 @@ public class BeamKafkaTable extends BaseBeamTable> { private Map configUpdates; protected BeamKafkaTable(RelProtoDataType protoRowType, - PTransform>, PCollection> sourceConverter, - PTransform, PCollection>> sinkConcerter) { + PTransform>, PCollection> sourceConverter, + PTransform, PCollection>> sinkConcerter) { super(protoRowType, sourceConverter, sinkConcerter); } public BeamKafkaTable(RelProtoDataType protoRowType, - PTransform>, PCollection> sourceConverter, - PTransform, PCollection>> sinkConcerter, + PTransform>, PCollection> sourceConverter, + PTransform, PCollection>> sinkConcerter, String bootstrapServers, List topics) { super(protoRowType, sourceConverter, sinkConcerter); this.bootstrapServers = bootstrapServers; @@ -57,37 +60,27 @@ public BeamIOType getSourceType() { } @Override - public PTransform>> buildReadTransform() { + public PTransform>> buildReadTransform() { return KafkaIO.read().withBootstrapServers(this.bootstrapServers) .withTopics(this.topics).updateConsumerProperties(configUpdates) - .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()); + .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()) + .withoutMetadata(); } @Override - public PTransform>, PDone> buildWriteTransform() { - checkArgument(topics != null && topics.size() == 0, + public PTransform>, PDone> buildWriteTransform() { + checkArgument(topics != null && topics.size() == 1, "Only one topic can be acceptable as output."); - return new PTransform>, PDone>() { + return new PTransform>, PDone>() { /** * */ private static final long serialVersionUID = 1136964183593770265L; @Override - public PDone expand(PCollection> input) { - return input.apply("toKafkaKV", - ParDo.of(new DoFn, KV>() { - /** - * - */ - private static final long serialVersionUID = 6265036192598208789L; - - @ProcessElement - public void processElement(ProcessContext ctx) { - ctx.output(ctx.element().getKV()); - } - })).apply("writeToKafka", + public PDone expand(PCollection> input) { + return input.apply("writeToKafka", KafkaIO.write().withBootstrapServers(bootstrapServers) .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) .withValueCoder(ByteArrayCoder.of())); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java new file mode 100644 index 000000000000..89ce000165bd --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java @@ -0,0 +1,36 @@ +package org.beam.sdk.java.sql.schema.kafka; + +import org.apache.beam.sdk.io.kafka.KafkaRecord; +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.DoFn.ProcessContext; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +public class KafkaKVExtractor extends PTransform>, PCollection>>{ + + /** + * + */ + private static final long serialVersionUID = 4802754456560670375L; + + @Override + public PCollection> expand(PCollection> input) { + return input.apply("toKafkaKV", + ParDo.of(new DoFn, KV>() { + /** + * + */ + private static final long serialVersionUID = 6265036192598208789L; + + @ProcessElement + public void processElement(ProcessContext ctx) { + ctx.output(ctx.element().getKV()); + } + })); + } + +} From 1272621c37ba76fd7a1997ba43f19d281e8030aa Mon Sep 17 00:00:00 2001 From: mingmxu Date: Fri, 10 Mar 2017 14:22:04 -0800 Subject: [PATCH 06/15] update license --- .../sdk/java/sql/examples/BeamSqlExample.java | 26 ++++++++++------ .../java/sql/examples/RheosSinkTransform.java | 17 +++++++++++ .../sql/examples/RheosSourceTransform.java | 17 +++++++++++ .../BeamSQLExpressionExecutor.java | 30 +++++++++++++++++++ .../sql/interpreter/BeamSQLSpELExecutor.java | 26 ++++++++++++---- .../java/sql/interpreter/CalciteToSpEL.java | 21 +++++++++++++ .../sdk/java/sql/interpreter/ProjectRule.java | 18 +++++++++++ .../sdk/java/sql/interpreter/ProjectType.java | 18 +++++++++++ .../java/sql/interpreter/package-info.java | 20 +++++++++++-- .../java/sql/planner/BeamPipelineCreator.java | 17 +++++++++++ .../java/sql/planner/BeamQueryPlanner.java | 17 +++++++++++ .../sql/planner/BeamRelDataTypeSystem.java | 17 +++++++++++ .../sdk/java/sql/planner/BeamRuleSets.java | 17 +++++++++++ .../sdk/java/sql/planner/BeamSQLRelUtils.java | 17 +++++++++++ .../beam/sdk/java/sql/rel/BeamFilterRel.java | 17 +++++++++++ .../beam/sdk/java/sql/rel/BeamIOSinkRel.java | 17 +++++++++++ .../sdk/java/sql/rel/BeamIOSourceRel.java | 17 +++++++++++ .../beam/sdk/java/sql/rel/BeamProjectRel.java | 17 +++++++++++ .../beam/sdk/java/sql/rel/BeamRelNode.java | 17 +++++++++++ .../beam/sdk/java/sql/rel/package-info.java | 18 +++++++++-- .../sdk/java/sql/rule/BeamFilterRule.java | 17 +++++++++++ .../sdk/java/sql/rule/BeamIOSinkRule.java | 17 +++++++++++ .../sdk/java/sql/rule/BeamIOSourceRule.java | 17 +++++++++++ .../beam/sdk/java/sql/rule/package-info.java | 20 +++++++++++-- .../sdk/java/sql/schema/BaseBeamTable.java | 17 +++++++++++ .../beam/sdk/java/sql/schema/BeamIOType.java | 17 +++++++++++ .../java/sql/schema/BeamSQLRecordType.java | 17 +++++++++++ .../beam/sdk/java/sql/schema/BeamSQLRow.java | 17 +++++++++++ .../java/sql/schema/kafka/BeamKafkaTable.java | 17 +++++++++++ .../schema/kafka/KafkaCSVRecordTransform.java | 17 +++++++++++ .../sql/schema/kafka/KafkaKVExtractor.java | 17 +++++++++++ .../java/sql/schema/kafka/package-info.java | 18 +++++++++-- .../java/sql/transform/BeamSQLFilterFn.java | 17 +++++++++++ .../transform/BeamSQLOutputToConsoleFn.java | 17 +++++++++++ .../java/sql/transform/BeamSQLProjectFn.java | 17 +++++++++++ .../sdk/java/sql/transform/package-info.java | 20 +++++++++++-- 36 files changed, 633 insertions(+), 27 deletions(-) diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java index 43018459d04f..2419f631d399 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.examples; import java.io.IOException; @@ -84,15 +101,6 @@ public RelDataType apply(RelDataTypeFactory a0) { .add("PAGEID", SqlTypeName.INTEGER).add("PAGENAME", SqlTypeName.VARCHAR).build(); } }; - Direction dir = Direction.ASCENDING; - RelFieldCollation collation = new RelFieldCollation(0, dir, NullDirection.UNSPECIFIED); - Statistic stat = Statistics.of(5, ImmutableList.of(ImmutableBitSet.of(0)), - ImmutableList.of(RelCollations.of(collation))); - - stat = Statistics.of(100d, ImmutableList.of(), - RelCollations.createSingleton(0)); - - stat = Statistics.UNKNOWN; Map consumerPara = new HashMap(); consumerPara.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); // latest diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java index c5c344895def..1ea00b1fc339 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.examples; import java.io.IOException; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java index 59e72ee7ef74..cc77a9306a80 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.examples; import java.io.IOException; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java index cfbfa6acabf4..3b2cb1c78dad 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.interpreter; import java.io.Serializable; @@ -5,10 +22,23 @@ import org.beam.sdk.java.sql.schema.BeamSQLRow; +/** + * {@code BeamSQLExpressionExecutor} fills the gap between relational expressions in Calcite SQL and executable code. + * + */ public interface BeamSQLExpressionExecutor extends Serializable { + /** + * invoked before data processing. + */ public void prepare(); + /** + * apply transformation to input record {@link BeamSQLRow}. + * + * @param inputRecord + * @return + */ public List execute(BeamSQLRow inputRecord); public void close(); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java index 4e27bedc3918..20d5cdfb3a30 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.interpreter; import java.util.ArrayList; @@ -18,6 +35,10 @@ import org.springframework.expression.spel.standard.SpelExpressionParser; import org.springframework.expression.spel.support.StandardEvaluationContext; +/** + * {@code BeamSQLSpELExecutor} is one implementation, to convert Calcite SQL relational expression to SpEL expression. + * + */ public class BeamSQLSpELExecutor implements BeamSQLExpressionExecutor { /** * @@ -36,12 +57,7 @@ public BeamSQLSpELExecutor(BeamRelNode relNode) { } else if (relNode instanceof BeamProjectRel) { List projectRules = createProjectRules((BeamProjectRel) relNode); for (int idx = 0; idx < projectRules.size(); ++idx) { -// if (projectRules.get(idx).getType().equals(ProjectType.RexCall) -// || projectRules.get(idx).getType().equals(ProjectType.RexLiteral)) { spelString.add(projectRules.get(idx).getProjectExp()); -// } else { -// spelString.add(null); // TODO -// } } } else { throw new BeamSqlUnsupportedException( diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java index 4514a7e9cd4f..186ca7845774 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.interpreter; import java.util.ArrayList; @@ -10,6 +27,10 @@ import com.google.common.base.Joiner; +/** + * {@code CalciteToSpEL} is used in {@link BeamSQLSpELExecutor}, to convert a relational expression {@link RexCall} to SpEL expression. + * + */ public class CalciteToSpEL { public static String rexcall2SpEL(RexCall cdn) { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java index 39b975a9aa54..bf475da97e0c 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java @@ -1,7 +1,25 @@ +/* + * 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.beam.sdk.java.sql.interpreter; import java.io.Serializable; +@Deprecated public class ProjectRule implements Serializable { /** * diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java index fcd1c51c89d0..abcf69c89701 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java @@ -1,7 +1,25 @@ +/* + * 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.beam.sdk.java.sql.interpreter; import java.io.Serializable; +@Deprecated public enum ProjectType implements Serializable { RexLiteral, RexInputRef, RexCall; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java index 244b243ea618..43d4651c94a2 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java @@ -1,8 +1,22 @@ -/** - * +/* + * 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. */ /** - * interpreter generate runnable 'code' to execute SQL operation, expressions. + * interpreter generate runnable 'code' to execute SQL relational expressions. * */ package org.beam.sdk.java.sql.interpreter; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java index cefdec5ab154..b619404ef19c 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.planner; import java.util.Map; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java index ca4724f06712..9efbb9a3af45 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.planner; import java.util.ArrayList; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java index 65dd473c579a..321e2a4c691e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.planner; import org.apache.calcite.rel.type.RelDataTypeSystem; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java index a4b1f6b3bd6c..a189cae414c2 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.planner; import java.util.Iterator; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java index 6de9ecb81f9e..88e028e2f941 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.planner; import java.util.concurrent.atomic.AtomicInteger; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java index fe0ff8ce034a..9d7c20554299 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rel; import org.apache.beam.sdk.transforms.ParDo; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java index cc6372bd2c59..01eb278300ec 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rel; import java.util.List; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java index 0fd4e13e7746..97b8d90f3d31 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rel; import org.apache.beam.sdk.values.PCollection; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java index 5f4d1f29c062..ee94684838d5 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rel; import java.util.List; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java index 67c7153f0f32..0f762ebe1c9d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rel; import org.apache.calcite.rel.RelNode; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java index e1e53d8edbdc..8e0df21e5e7a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java @@ -1,5 +1,19 @@ -/** - * +/* + * 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. */ /** * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java index 37afbb56c4a7..ee8185a9de1e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rule; import org.apache.calcite.plan.Convention; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java index bc75d4bd855c..b4eb1daa4be6 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rule; import java.util.List; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java index 1465133dd5b6..5e62e8021658 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.rule; import org.apache.calcite.plan.Convention; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java index bfce4981fb6f..502bdc70cfab 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java @@ -1,8 +1,22 @@ -/** - * +/* + * 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. */ /** - * @author mingmxu + * * */ package org.beam.sdk.java.sql.rule; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java index 67c3608db78b..39400dfb4461 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.schema; import java.io.Serializable; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java index ad9bb84007f4..9079cb92ada6 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.schema; import java.io.Serializable; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java index 85cd180f9e06..c3dffbc7948d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.schema; import java.io.Serializable; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java index 018ec9f6040e..30010834460d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.schema; import java.io.Serializable; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index 773e7f3d375e..d5e8e3ced92b 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.schema.kafka; import static com.google.common.base.Preconditions.checkArgument; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java index 9edb03f2650e..321fa748b32b 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.schema.kafka; import org.apache.beam.sdk.io.kafka.KafkaRecord; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java index 89ce000165bd..6271ecf3ad92 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.schema.kafka; import org.apache.beam.sdk.io.kafka.KafkaRecord; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java index 75cb2853451d..0c0ce350417a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java @@ -1,5 +1,19 @@ -/** - * +/* + * 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. */ /** * table schema for KafkaIO diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java index 7158621fdace..60aa1d644451 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.transform; import java.util.List; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java index 094973208d8f..c78fbe2b1209 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.transform; import org.apache.beam.sdk.transforms.DoFn; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java index 07e3faa4c038..a2d0a93724a6 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.transform; import java.util.List; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java index de6377db41d1..8548a4a716b3 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java @@ -1,8 +1,22 @@ -/** - * +/* + * 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. */ /** - * @author mingmxu + * * */ package org.beam.sdk.java.sql.transform; \ No newline at end of file From e11dfcf9e1b0aae0eb26ebd2246a914957b8e712 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Fri, 10 Mar 2017 17:45:46 -0800 Subject: [PATCH 07/15] save work: add BeamSqlRunner as the tier1 interface; --- sdks/java/sql/pom.xml | 8 ++ .../sdk/java/sql/examples/BeamSqlExample.java | 53 +++----- .../java/sql/planner/BeamPipelineCreator.java | 8 -- .../java/sql/planner/BeamQueryPlanner.java | 70 +++++++--- .../sdk/java/sql/planner/BeamSqlRunner.java | 120 ++++++++++++++++++ .../beam/sdk/java/sql/rel/BeamFilterRel.java | 5 +- .../beam/sdk/java/sql/rel/BeamIOSinkRel.java | 9 +- .../sdk/java/sql/rel/BeamIOSourceRel.java | 9 +- .../beam/sdk/java/sql/rel/BeamProjectRel.java | 5 +- .../beam/sdk/java/sql/rel/BeamRelNode.java | 3 +- .../sdk/java/sql/schema/BaseBeamTable.java | 42 +++--- .../java/sql/schema/kafka/BeamKafkaTable.java | 21 +-- .../sql/schema/kafka/KafkaKVExtractor.java | 53 -------- .../sql/src/main/resources/log4j.properties | 23 ++++ .../beam/sdk/java/sql/e2e/package-info.java | 22 ++++ 15 files changed, 288 insertions(+), 163 deletions(-) create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java create mode 100644 sdks/java/sql/src/main/resources/log4j.properties create mode 100644 sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java diff --git a/sdks/java/sql/pom.xml b/sdks/java/sql/pom.xml index 9dff6ab3e4d4..a22300fd8cdc 100644 --- a/sdks/java/sql/pom.xml +++ b/sdks/java/sql/pom.xml @@ -175,6 +175,14 @@ rheos-client 0.0.4 + + org.slf4j + slf4j-jdk14 + + + org.slf4j + slf4j-api + diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java index 2419f631d399..0e39d4da634d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java @@ -44,55 +44,35 @@ import org.apache.calcite.util.ImmutableBitSet; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.beam.sdk.java.sql.planner.BeamQueryPlanner; +import org.beam.sdk.java.sql.planner.BeamSqlRunner; +import org.beam.sdk.java.sql.schema.BaseBeamTable; import org.beam.sdk.java.sql.schema.BeamSQLRecordType; import org.beam.sdk.java.sql.schema.kafka.BeamKafkaTable; import com.google.common.collect.ImmutableList; public class BeamSqlExample implements Serializable { - private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); - private final SchemaPlus schema = Frameworks.createRootSchema(true); - public void addSchema(String schemaName, Schema scheme) { - schema.add(schemaName, schema); - } - - public void addTable(String tableName, Table table) { - schema.add(tableName, table); - } - - public void explainAndRun(String sqlString) throws IOException, SQLException { - - BeamQueryPlanner planner = new BeamQueryPlanner(schema); - try { - System.out.println("SQL>: \n" + sqlString); - planner.compileAndRun(sqlString); - } catch (Exception e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - - public static void main(String[] args) throws IOException, SQLException { - BeamSqlExample runner = new BeamSqlExample(); - runner.initTables(); + public static void main(String[] args) throws Exception { + BeamSqlRunner runner = new BeamSqlRunner(); + runner.addTable("SOJ_EVENT", + getTable("SOJ_EVENT", + "rheos-kafka-proxy-1.lvs02.dev.ebayc3.com:9092,rheos-kafka-proxy-2.lvs02.dev.ebayc3.com:9092", + "behavior.pulsar.sojevent.total")); + runner.addTable("subrowverevent", + getTable("subrowverevent", "flink2-8332.lvs01.dev.ebayc3.com:9092", "subrowverevent")); // case 2: insert into
() select STREAM from //
from String sql = "INSERT INTO subrowverevent(SITEID, PAGEID) " + "SELECT " + " SITEID, PAGEID as new_pageId " + "FROM SOJ_EVENT " + "WHERE SITEID > 0 "; - runner.explainAndRun(sql); - } - - void initTables() { - addTable("SOJ_EVENT", - "rheos-kafka-proxy-1.lvs02.dev.ebayc3.com:9092,rheos-kafka-proxy-2.lvs02.dev.ebayc3.com:9092", - "behavior.pulsar.sojevent.total"); - addTable("subrowverevent", "flink2-8332.lvs01.dev.ebayc3.com:9092", "subrowverevent"); + runner.explainQuery(sql); + System.out.println("**********"); + runner.submitQuery(sql); } - void addTable(String tableName, String bootstrapServer, String topic) { + public static BaseBeamTable getTable(String tableName, String bootstrapServer, String topic) { final RelProtoDataType protoRowType = new RelProtoDataType() { @Override public RelDataType apply(RelDataTypeFactory a0) { @@ -109,12 +89,11 @@ public RelDataType apply(RelDataTypeFactory a0) { consumerPara.put(ConsumerConfig.GROUP_ID_CONFIG, "toraframeworktest_sojeventproxy"); consumerPara.put(ConsumerConfig.CLIENT_ID_CONFIG, "107b877c-2694-4298-beed-9007f54602aa"); - addTable(tableName, - new BeamKafkaTable(protoRowType, + return new BeamKafkaTable(protoRowType, new RheosSourceTransform( BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), new RheosSinkTransform( BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), - bootstrapServer, Arrays.asList(topic)).updateConsumerProperties(consumerPara)); + bootstrapServer, Arrays.asList(topic)).updateConsumerProperties(consumerPara); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java index b619404ef19c..3d7f555cd32f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java @@ -59,14 +59,6 @@ public BeamPipelineCreator(Map sourceTables) { pipeline = Pipeline.create(options); } - public void runJob() { - if (!hasPersistent) { - latestStream.apply("emit_to_console", ParDo.of(new BeamSQLOutputToConsoleFn("emit_to_console"))); - } - - pipeline.run(); - } - public PCollection getLatestStream() { return latestStream; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java index 9efbb9a3af45..941ec55b6e0e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.config.Lex; import org.apache.calcite.jdbc.CalciteSchema; @@ -52,18 +54,20 @@ import org.beam.sdk.java.sql.schema.BaseBeamTable; /** - * The core component to handle through a SQL statement, to a Beam pipeline. + * The core component to handle through a SQL statement, to submit a Beam pipeline. * */ public class BeamQueryPlanner { - public static final int BEAM_REL_CONVERSION_RULES = 1; - - private final Planner planner; - private Map kafkaTables = new HashMap<>(); + protected final Planner planner; + private Map sourceTables = new HashMap<>(); private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + /** + * + * @param schema + */ public BeamQueryPlanner(SchemaPlus schema) { final List traitDefs = new ArrayList(); traitDefs.add(ConventionTraitDef.INSTANCE); @@ -83,27 +87,48 @@ public BeamQueryPlanner(SchemaPlus schema) { this.planner = Frameworks.getPlanner(config); for (String t : schema.getTableNames()) { - kafkaTables.put(t, (BaseBeamTable) schema.getTable(t)); + sourceTables.put(t, (BaseBeamTable) schema.getTable(t)); } } - public void compileAndRun(String query) throws Exception { - BeamRelNode relNode = getPlan(query); - - BeamPipelineCreator planCreator = new BeamPipelineCreator(kafkaTables); - - String beamPlan = RelOptUtil.toString(relNode); - System.out.println("beamPlan>"); - System.out.println(beamPlan); - - relNode.buildBeamPipeline(planCreator); - - planCreator.runJob(); + /** + * With a Beam pipeline generated in {@link #compileBeamPipeline(String)}, submit it to run and wait until finish. + * + * @param sqlStatement + * @throws Exception + */ + public void submitToRun(String sqlStatement) throws Exception{ + Pipeline pipeline = compileBeamPipeline(sqlStatement); + + PipelineResult result = pipeline.run(); + result.waitUntilFinish(); + } + + /** + * With the @{@link BeamRelNode} tree generated in {@link #convertToBeamRel(String)}, a Beam pipeline is generated. + * + * @param sqlStatement + * @return + * @throws Exception + */ + public Pipeline compileBeamPipeline(String sqlStatement) throws Exception { + BeamRelNode relNode = convertToBeamRel(sqlStatement); + + BeamPipelineCreator planCreator = new BeamPipelineCreator(sourceTables); + return relNode.buildBeamPipeline(planCreator); } - public BeamRelNode getPlan(String query) + /** + * It parses and validate the input query, then convert into a {@link BeamRelNode} tree. + * @param query + * @return + * @throws ValidationException + * @throws RelConversionException + * @throws SqlParseException + */ + public BeamRelNode convertToBeamRel(String sqlStatement) throws ValidationException, RelConversionException, SqlParseException { - return (BeamRelNode) validateAndConvert(planner.parse(query)); + return (BeamRelNode) validateAndConvert(planner.parse(sqlStatement)); } private RelNode validateAndConvert(SqlNode sqlNode) @@ -133,4 +158,9 @@ private SqlNode validateNode(SqlNode sqlNode) throws ValidationException { return validatedSqlNode; } + public Map getSourceTables() { + return sourceTables; + } + + } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java new file mode 100644 index 000000000000..c274e013df0c --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java @@ -0,0 +1,120 @@ +/* + * 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.beam.sdk.java.sql.planner; + +import java.io.IOException; +import java.io.Serializable; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import org.apache.calcite.adapter.java.JavaTypeFactory; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.RelFieldCollation; +import org.apache.calcite.rel.RelFieldCollation.Direction; +import org.apache.calcite.rel.RelFieldCollation.NullDirection; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.schema.Schema; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.schema.Statistic; +import org.apache.calcite.schema.Statistics; +import org.apache.calcite.schema.Table; +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.tools.RelConversionException; +import org.apache.calcite.tools.ValidationException; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.beam.sdk.java.sql.planner.BeamQueryPlanner; +import org.beam.sdk.java.sql.rel.BeamRelNode; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; +import org.beam.sdk.java.sql.schema.kafka.BeamKafkaTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.ImmutableList; + +public class BeamSqlRunner implements Serializable { + /** + * + */ + private static final long serialVersionUID = -4708693435115005182L; + + private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRunner.class); + + private JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + private SchemaPlus schema = Frameworks.createRootSchema(true); + + private BeamQueryPlanner planner = new BeamQueryPlanner(schema); + + /** + * Add a schema. + * + * @param schemaName + * @param scheme + */ + public void addSchema(String schemaName, Schema scheme) { + schema.add(schemaName, schema); + } + + /** + * add a {@link BaseBeamTable} to schema repository. + * + * @param tableName + * @param table + */ + public void addTable(String tableName, BaseBeamTable table) { + schema.add(tableName, table); + planner.getSourceTables().put(tableName, table); + } + + /** + * submit as a Beam pipeline. + * + * @param sqlString + * @throws Exception + */ + public void submitQuery(String sqlString) throws Exception{ + planner.submitToRun(sqlString); + planner.planner.close(); + } + + /** + * explain and display the execution plan. + * + * @param sqlString + * @throws ValidationException + * @throws RelConversionException + * @throws SqlParseException + */ + public void explainQuery(String sqlString) throws ValidationException, RelConversionException, SqlParseException { + BeamRelNode exeTree = planner.convertToBeamRel(sqlString); + String beamPlan = RelOptUtil.toString(exeTree); + LOG.info("beamPlan>"); + LOG.info(beamPlan); + planner.planner.close(); + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java index 9d7c20554299..e8f311f7e92d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java @@ -17,6 +17,7 @@ */ package org.beam.sdk.java.sql.rel; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; @@ -44,7 +45,7 @@ public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) { } @Override - public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { RelNode input = getInput(); BeamSQLRelUtils.getBeamRelInput(input).buildBeamPipeline(planCreator); @@ -60,7 +61,7 @@ public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception planCreator.setLatestStream(projectStream); - System.out.println("Build: apply_filter " + stageName ); + return planCreator.getPipeline(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java index 01eb278300ec..a671d5a661d3 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java @@ -19,6 +19,7 @@ import java.util.List; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.kafka.KafkaIO; @@ -54,7 +55,7 @@ public RelNode copy(RelTraitSet traitSet, List inputs) { } @Override - public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { RelNode input = getInput(); BeamSQLRelUtils.getBeamRelInput(input).buildBeamPipeline(planCreator); @@ -67,12 +68,12 @@ public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception BaseBeamTable targetTable = planCreator.getSourceTables().get(sourceName); - PCollection formattedOutput = upstream.apply("preformat_to_target", targetTable.getSinkConcerter()); - formattedOutput.apply("persistent", targetTable.buildWriteTransform()); + PCollection formattedOutput = upstream.apply("preformat_to_target", targetTable.getOutputTransform()); + formattedOutput.apply("persistent", targetTable.buildIOWriter()); planCreator.setHasPersistent(true); - System.out.println("Build: add_persistent " + stageName); + return planCreator.getPipeline(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java index 97b8d90f3d31..760e78987b6f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java @@ -17,6 +17,7 @@ */ package org.beam.sdk.java.sql.rel; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; @@ -36,7 +37,7 @@ public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable } @Override - public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { String sourceName = Joiner.on('.').join(getTable().getQualifiedName()).replace(".(STREAM)", ""); @@ -45,12 +46,12 @@ public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception String stageName = BeamSQLRelUtils.getStageName(this); PCollection sourceStream = planCreator.getPipeline() - .apply(stageName, sourceTable.buildReadTransform()); - PCollection reformattedSourceStream = sourceStream.apply("sourceReformat", sourceTable.getSourceConverter()); + .apply(stageName, sourceTable.buildIOReader()); + PCollection reformattedSourceStream = sourceStream.apply("sourceReformat", sourceTable.getInputTransform()); planCreator.setLatestStream(reformattedSourceStream); - System.out.println("Build: add_source " + sourceName); + return planCreator.getPipeline(); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java index ee94684838d5..34ce54da91bc 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java @@ -19,6 +19,7 @@ import java.util.List; +import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; @@ -61,7 +62,7 @@ public Project copy(RelTraitSet traitSet, RelNode input, List projects, } @Override - public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { + public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { RelNode input = getInput(); BeamSQLRelUtils.getBeamRelInput(input).buildBeamPipeline(planCreator); @@ -76,7 +77,7 @@ public void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception planCreator.setLatestStream(projectStream); - System.out.println("Build: apply_project " + stageName); + return planCreator.getPipeline(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java index 0f762ebe1c9d..78ebf03c9d17 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java @@ -17,6 +17,7 @@ */ package org.beam.sdk.java.sql.rel; +import org.apache.beam.sdk.Pipeline; import org.apache.calcite.rel.RelNode; import org.beam.sdk.java.sql.planner.BeamPipelineCreator; @@ -31,5 +32,5 @@ public interface BeamRelNode extends RelNode { * @param planCreator * @throws Exception */ - void buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception; + Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java index 39400dfb4461..fcc5ba55e2ea 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java @@ -43,17 +43,17 @@ public abstract class BaseBeamTable implements ScannableTable, Serializable { */ private static final long serialVersionUID = -1262988061830914193L; private RelProtoDataType protoRowType; - // A transform to convert from a rawRecord of input - private PTransform, PCollection> sourceConverter; - // A transform to convert one record to a rawRecord for output - private PTransform, PCollection> sinkConcerter; + // A {@link PTransform} that convert a input record of {@code IO.read()} to {@link BeamSQLRow} + private PTransform, PCollection> inputTransform; + // A {@link PTransform} that convert a {@link BeamSQLRow} to the required record of {@code IO.write()} + private PTransform, PCollection> outputTransform; public BaseBeamTable(RelProtoDataType protoRowType, - PTransform, PCollection> sourceConverter, - PTransform, PCollection> sinkConcerter) { + PTransform, PCollection> inputTransform, + PTransform, PCollection> outputTransform) { this.protoRowType = protoRowType; - this.sourceConverter = sourceConverter; - this.sinkConcerter = sinkConcerter; + this.inputTransform = inputTransform; + this.outputTransform = outputTransform; } /** @@ -63,25 +63,35 @@ public BaseBeamTable(RelProtoDataType protoRowType, public abstract BeamIOType getSourceType(); /** - * create a READ PTransform. + * create a {@code IO.read()} instance to read from source. * * @return */ - public abstract PTransform> buildReadTransform(); + public abstract PTransform> buildIOReader(); /** - * create a WRITE PTransform + * create a {@code IO.write()} instance to write to target. * * @return */ - public abstract PTransform, PDone> buildWriteTransform(); + public abstract PTransform, PDone> buildIOWriter(); - public PTransform, PCollection> getSourceConverter() { - return sourceConverter; + + + /** + * A {@link PTransform} that convert a input record of {@code IO.read()} to {@link BeamSQLRow} + * @return + */ + public PTransform, PCollection> getInputTransform() { + return inputTransform; } - public PTransform, PCollection> getSinkConcerter() { - return sinkConcerter; + /** + * A {@link PTransform} that convert a {@link BeamSQLRow} to the required record of {@code IO.write()} + * @return + */ + public PTransform, PCollection> getOutputTransform() { + return outputTransform; } @Override diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index d5e8e3ced92b..846e156a8301 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -77,7 +77,7 @@ public BeamIOType getSourceType() { } @Override - public PTransform>> buildReadTransform() { + public PTransform>> buildIOReader() { return KafkaIO.read().withBootstrapServers(this.bootstrapServers) .withTopics(this.topics).updateConsumerProperties(configUpdates) .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()) @@ -85,24 +85,13 @@ public PTransform>> buildReadTran } @Override - public PTransform>, PDone> buildWriteTransform() { + public PTransform>, PDone> buildIOWriter() { checkArgument(topics != null && topics.size() == 1, "Only one topic can be acceptable as output."); - return new PTransform>, PDone>() { - /** - * - */ - private static final long serialVersionUID = 1136964183593770265L; - - @Override - public PDone expand(PCollection> input) { - return input.apply("writeToKafka", - KafkaIO.write().withBootstrapServers(bootstrapServers) - .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) - .withValueCoder(ByteArrayCoder.of())); - } - }; + return KafkaIO.write().withBootstrapServers(bootstrapServers) + .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) + .withValueCoder(ByteArrayCoder.of()); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java deleted file mode 100644 index 6271ecf3ad92..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaKVExtractor.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.beam.sdk.java.sql.schema.kafka; - -import org.apache.beam.sdk.io.kafka.KafkaRecord; -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.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.beam.sdk.java.sql.schema.BeamSQLRow; - -public class KafkaKVExtractor extends PTransform>, PCollection>>{ - - /** - * - */ - private static final long serialVersionUID = 4802754456560670375L; - - @Override - public PCollection> expand(PCollection> input) { - return input.apply("toKafkaKV", - ParDo.of(new DoFn, KV>() { - /** - * - */ - private static final long serialVersionUID = 6265036192598208789L; - - @ProcessElement - public void processElement(ProcessContext ctx) { - ctx.output(ctx.element().getKV()); - } - })); - } - -} diff --git a/sdks/java/sql/src/main/resources/log4j.properties b/sdks/java/sql/src/main/resources/log4j.properties new file mode 100644 index 000000000000..709484b4951b --- /dev/null +++ b/sdks/java/sql/src/main/resources/log4j.properties @@ -0,0 +1,23 @@ +################################################################################ +# 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. +################################################################################ + +log4j.rootLogger=ERROR,console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n \ No newline at end of file diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java new file mode 100644 index 000000000000..5e20fb3d6dad --- /dev/null +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ +/** + * contain end-to-end tests + * + */ +package org.beam.sdk.java.sql.e2e; \ No newline at end of file From 1d80bb5b7813949ca3bb05fd92bdd706d4e43084 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Fri, 10 Mar 2017 23:22:25 -0800 Subject: [PATCH 08/15] remove test code; --- "sdks/java/sql/README.md\n" | 24 ++++ sdks/java/sql/pom.xml | 40 +++++-- .../sdk/java/sql/examples/BeamSqlExample.java | 99 ----------------- .../java/sql/examples/RheosSinkTransform.java | 57 ---------- .../sql/examples/RheosSourceTransform.java | 103 ------------------ .../BeamSQLExpressionExecutor.java | 3 +- .../sql/interpreter/BeamSQLSpELExecutor.java | 46 ++++---- .../java/sql/interpreter/CalciteToSpEL.java | 16 +-- .../sdk/java/sql/interpreter/ProjectRule.java | 66 ----------- .../sdk/java/sql/interpreter/ProjectType.java | 25 ----- .../org/beam/sdk/java/sql/package-info.java | 3 +- .../java/sql/planner/BeamPipelineCreator.java | 11 +- .../java/sql/planner/BeamQueryPlanner.java | 31 +++--- .../sdk/java/sql/planner/BeamRuleSets.java | 15 ++- .../sdk/java/sql/planner/BeamSqlRunner.java | 39 ++----- .../sdk/java/sql/planner/package-info.java | 4 +- .../beam/sdk/java/sql/rel/BeamFilterRel.java | 3 +- .../beam/sdk/java/sql/rel/BeamIOSinkRel.java | 15 +-- .../sdk/java/sql/rel/BeamIOSourceRel.java | 8 +- .../beam/sdk/java/sql/rel/BeamProjectRel.java | 8 +- .../beam/sdk/java/sql/rel/BeamRelNode.java | 10 +- .../sdk/java/sql/rule/BeamProjectRule.java | 3 +- .../sdk/java/sql/schema/BaseBeamTable.java | 16 ++- .../beam/sdk/java/sql/schema/BeamSQLRow.java | 10 +- .../java/sql/schema/kafka/BeamKafkaTable.java | 12 +- .../schema/kafka/KafkaCSVRecordTransform.java | 57 ---------- .../java/sql/transform/BeamSQLFilterFn.java | 6 +- .../java/sql/transform/BeamSQLProjectFn.java | 13 ++- 28 files changed, 177 insertions(+), 566 deletions(-) create mode 100644 "sdks/java/sql/README.md\n" delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java diff --git "a/sdks/java/sql/README.md\n" "b/sdks/java/sql/README.md\n" new file mode 100644 index 000000000000..ae9e0f3f6fd5 --- /dev/null +++ "b/sdks/java/sql/README.md\n" @@ -0,0 +1,24 @@ + + +# Beam SQL + +Beam SQL provides a new interface, to execute a SQL query as a Beam pipeline. + +*It's working in progress...* diff --git a/sdks/java/sql/pom.xml b/sdks/java/sql/pom.xml index a22300fd8cdc..d322228aae2f 100644 --- a/sdks/java/sql/pom.xml +++ b/sdks/java/sql/pom.xml @@ -67,6 +67,35 @@ ${project.basedir}/src/test/ + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + + org.apache.maven.plugins + + + maven-checkstyle-plugin + + [2.17,) + + check + + + + + + + + + + @@ -155,12 +184,6 @@ org.apache.beam beam-sdks-java-io-kafka - - - org.apache.kafka - kafka-clients - - org.springframework @@ -170,11 +193,6 @@ com.google.guava guava - - io.ebay.rheos - rheos-client - 0.0.4 - org.slf4j slf4j-jdk14 diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java deleted file mode 100644 index 0e39d4da634d..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/BeamSqlExample.java +++ /dev/null @@ -1,99 +0,0 @@ -/* - * 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.beam.sdk.java.sql.examples; - -import java.io.IOException; -import java.io.Serializable; -import java.sql.SQLException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; - -import org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.calcite.jdbc.JavaTypeFactoryImpl; -import org.apache.calcite.rel.RelCollations; -import org.apache.calcite.rel.RelFieldCollation; -import org.apache.calcite.rel.RelFieldCollation.Direction; -import org.apache.calcite.rel.RelFieldCollation.NullDirection; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.rel.type.RelProtoDataType; -import org.apache.calcite.schema.Schema; -import org.apache.calcite.schema.SchemaPlus; -import org.apache.calcite.schema.Statistic; -import org.apache.calcite.schema.Statistics; -import org.apache.calcite.schema.Table; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.tools.Frameworks; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.beam.sdk.java.sql.planner.BeamQueryPlanner; -import org.beam.sdk.java.sql.planner.BeamSqlRunner; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import org.beam.sdk.java.sql.schema.kafka.BeamKafkaTable; - -import com.google.common.collect.ImmutableList; - -public class BeamSqlExample implements Serializable { - - public static void main(String[] args) throws Exception { - BeamSqlRunner runner = new BeamSqlRunner(); - runner.addTable("SOJ_EVENT", - getTable("SOJ_EVENT", - "rheos-kafka-proxy-1.lvs02.dev.ebayc3.com:9092,rheos-kafka-proxy-2.lvs02.dev.ebayc3.com:9092", - "behavior.pulsar.sojevent.total")); - runner.addTable("subrowverevent", - getTable("subrowverevent", "flink2-8332.lvs01.dev.ebayc3.com:9092", "subrowverevent")); - - // case 2: insert into
() select STREAM from - //
from - String sql = "INSERT INTO subrowverevent(SITEID, PAGEID) " + "SELECT " - + " SITEID, PAGEID as new_pageId " + "FROM SOJ_EVENT " + "WHERE SITEID > 0 "; - - runner.explainQuery(sql); - System.out.println("**********"); - runner.submitQuery(sql); - } - - public static BaseBeamTable getTable(String tableName, String bootstrapServer, String topic) { - final RelProtoDataType protoRowType = new RelProtoDataType() { - @Override - public RelDataType apply(RelDataTypeFactory a0) { - return a0.builder().add("EVENTTIMESTAMP", SqlTypeName.TIMESTAMP) - .add("ITEMID", SqlTypeName.BIGINT).add("SITEID", SqlTypeName.INTEGER) - .add("PAGEID", SqlTypeName.INTEGER).add("PAGENAME", SqlTypeName.VARCHAR).build(); - } - }; - - Map consumerPara = new HashMap(); - consumerPara.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); // latest - // or - // earliest - consumerPara.put(ConsumerConfig.GROUP_ID_CONFIG, "toraframeworktest_sojeventproxy"); - consumerPara.put(ConsumerConfig.CLIENT_ID_CONFIG, "107b877c-2694-4298-beed-9007f54602aa"); - - return new BeamKafkaTable(protoRowType, - new RheosSourceTransform( - BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), - new RheosSinkTransform( - BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl()))), - bootstrapServer, Arrays.asList(topic)).updateConsumerProperties(consumerPara); - } -} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java deleted file mode 100644 index 1ea00b1fc339..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSinkTransform.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.beam.sdk.java.sql.examples; - -import java.io.IOException; - -import org.apache.beam.sdk.io.kafka.KafkaRecord; -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.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import org.beam.sdk.java.sql.schema.BeamSQLRow; - -public class RheosSinkTransform extends PTransform, PCollection>>{ - /** - * - */ - private static final long serialVersionUID = 6589679229923907701L; - private BeamSQLRecordType recordType; - - - public RheosSinkTransform(BeamSQLRecordType recordType) { - super(); - this.recordType = recordType; - } - - - @Override - public PCollection> expand(PCollection input) { - return input.apply("toKafkaRecord", ParDo.of(new DoFn>(){ - @ProcessElement - public void processElement(ProcessContext c) throws IOException { - c.output(KV.of(new byte[]{}, c.element().getDataMap().toString().getBytes())); - } - })); - } - -} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java deleted file mode 100644 index cc77a9306a80..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/RheosSourceTransform.java +++ /dev/null @@ -1,103 +0,0 @@ -/* - * 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.beam.sdk.java.sql.examples; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -import org.apache.avro.Schema.Field; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.Decoder; -import org.apache.avro.io.DecoderFactory; -import org.apache.beam.sdk.io.kafka.KafkaRecord; -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.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import org.beam.sdk.java.sql.schema.BeamSQLRow; - -import io.ebay.rheos.kafka.client.StreamConnectorConfig; -import io.ebay.rheos.schema.avro.RheosEventDeserializer; -import io.ebay.rheos.schema.avro.SchemaRegistryAwareAvroDeserializerHelper; -import io.ebay.rheos.schema.event.RheosEvent; - -public class RheosSourceTransform extends PTransform>, PCollection>{ - /** - * - */ - private static final long serialVersionUID = -7803885128685230359L; - private BeamSQLRecordType recordType; - - - public RheosSourceTransform(BeamSQLRecordType recordType) { - super(); - this.recordType = recordType; - } - - - @Override - public PCollection expand(PCollection> input) { - return input.apply("soureDecode", ParDo.of(new DoFn, BeamSQLRow>(){ - transient RheosEventDeserializer rheosDeserializer; - transient SchemaRegistryAwareAvroDeserializerHelper deserializerHelper; - - @Setup - public void setup() { - this.rheosDeserializer = new RheosEventDeserializer(); - - Map config = new HashMap<>(); - config.put(StreamConnectorConfig.RHEOS_SERVICES_URLS, "https://rheos-services.qa.ebay.com"); - - deserializerHelper = new SchemaRegistryAwareAvroDeserializerHelper<>(config, - GenericRecord.class); - } - - @ProcessElement - public void processElement(ProcessContext c) throws IOException { - byte[] rawBytes = c.element().getValue(); - RheosEvent event = rheosDeserializer.deserialize("", rawBytes); - - DatumReader reader = new GenericDatumReader( - deserializerHelper.getSchema(event.getSchemaId())); - Decoder decoder = DecoderFactory.get().binaryDecoder(event.toBytes(), null); - - GenericRecord record = reader.read(null, decoder); - Map k2k = new HashMap<>(); - for (Field f : record.getSchema().getFields()) { - k2k.put(f.name().toUpperCase(), f.name()); - } - -// BeamSQLRecordType recordType = BeamSQLRecordType.from(protoRowType.apply(new JavaTypeFactoryImpl())); - BeamSQLRow values = new BeamSQLRow(recordType); - for (int idx = 0; idx < recordType.getFieldsName().size(); ++idx) { - values.addField(recordType.getFieldsName().get(idx), - record.get(k2k.get(recordType.getFieldsName().get(idx)))); - } - - c.output(values); - } - - })); - } - -} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java index 3b2cb1c78dad..24785569da21 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java @@ -23,7 +23,8 @@ import org.beam.sdk.java.sql.schema.BeamSQLRow; /** - * {@code BeamSQLExpressionExecutor} fills the gap between relational expressions in Calcite SQL and executable code. + * {@code BeamSQLExpressionExecutor} fills the gap between relational + * expressions in Calcite SQL and executable code. * */ public interface BeamSQLExpressionExecutor extends Serializable { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java index 20d5cdfb3a30..6578fb912626 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java @@ -17,6 +17,8 @@ */ package org.beam.sdk.java.sql.interpreter; +import static com.google.common.base.Preconditions.checkArgument; + import java.util.ArrayList; import java.util.List; @@ -36,7 +38,8 @@ import org.springframework.expression.spel.support.StandardEvaluationContext; /** - * {@code BeamSQLSpELExecutor} is one implementation, to convert Calcite SQL relational expression to SpEL expression. + * {@code BeamSQLSpELExecutor} is one implementation, to convert Calcite SQL + * relational expression to SpEL expression. * */ public class BeamSQLSpELExecutor implements BeamSQLExpressionExecutor { @@ -55,10 +58,11 @@ public BeamSQLSpELExecutor(BeamRelNode relNode) { .rexcall2SpEL((RexCall) ((BeamFilterRel) relNode).getCondition()); spelString.add(filterSpEL); } else if (relNode instanceof BeamProjectRel) { - List projectRules = createProjectRules((BeamProjectRel) relNode); - for (int idx = 0; idx < projectRules.size(); ++idx) { - spelString.add(projectRules.get(idx).getProjectExp()); - } + spelString.addAll(createProjectExps((BeamProjectRel) relNode)); + // List projectRules = + // for (int idx = 0; idx < projectRules.size(); ++idx) { + // spelString.add(projectRules.get(idx).getProjectExp()); + // } } else { throw new BeamSqlUnsupportedException( String.format("%s is not supported yet", relNode.getClass().toString())); @@ -93,33 +97,31 @@ public void close() { } - private List createProjectRules(BeamProjectRel projectRel) { - List rules = new ArrayList<>(); + private List createProjectExps(BeamProjectRel projectRel) { + List rules = new ArrayList<>(); List exps = projectRel.getProjects(); for (int idx = 0; idx < exps.size(); ++idx) { RexNode node = exps.get(idx); - ProjectRule rule = new ProjectRule(); + if (node == null) { + rules.add("null"); + } if (node instanceof RexLiteral) { - rule.setType(ProjectType.RexLiteral); - rule.setProjectExp(((RexLiteral) node).getValue() + ""); - }else{ - - if (node instanceof RexInputRef) { - rule.setType(ProjectType.RexInputRef); -// rule.setSourceIndex( ((RexInputRef) node).getIndex() ); - rule.setProjectExp("#in.getFieldValue("+((RexInputRef) node).getIndex()+")"); + rules.add(((RexLiteral) node).getValue() + ""); + } else { + if (node instanceof RexInputRef) { + rules.add("#in.getFieldValue(" + ((RexInputRef) node).getIndex() + ")"); + } + if (node instanceof RexCall) { + rules.add(CalciteToSpEL.rexcall2SpEL((RexCall) node)); + } } - if (node instanceof RexCall) { - rule.setType(ProjectType.RexCall); - rule.setProjectExp(CalciteToSpEL.rexcall2SpEL((RexCall) node)); - } - rules.add(rule); - } } + checkArgument(rules.size() == exps.size(), "missing projects rules after conversion."); + return rules; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java index 186ca7845774..aac23cbe734a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java @@ -28,7 +28,8 @@ import com.google.common.base.Joiner; /** - * {@code CalciteToSpEL} is used in {@link BeamSQLSpELExecutor}, to convert a relational expression {@link RexCall} to SpEL expression. + * {@code CalciteToSpEL} is used in {@link BeamSQLSpELExecutor}, to convert a + * relational expression {@link RexCall} to SpEL expression. * */ public class CalciteToSpEL { @@ -40,8 +41,7 @@ public static String rexcall2SpEL(RexCall cdn) { parts.add(rexcall2SpEL((RexCall) subcdn)); } else { parts.add(subcdn instanceof RexInputRef - ? "#in.getFieldValue("+ ((RexInputRef) subcdn).getIndex() + ")" - : subcdn.toString()); + ? "#in.getFieldValue(" + ((RexInputRef) subcdn).getIndex() + ")" : subcdn.toString()); } } @@ -51,15 +51,15 @@ public static String rexcall2SpEL(RexCall cdn) { case "SqlBinaryOperator": // > < = >= <= <> OR AND || / . switch (cdn.op.getName().toUpperCase()) { case "AND": - return String.format(" ( %s ) ", Joiner.on("&&").join(parts) ); + return String.format(" ( %s ) ", Joiner.on("&&").join(parts)); case "OR": - return String.format(" ( %s ) ", Joiner.on("||").join(parts) ); + return String.format(" ( %s ) ", Joiner.on("||").join(parts)); case "=": - return String.format(" ( %s ) ", Joiner.on("==").join(parts) ); + return String.format(" ( %s ) ", Joiner.on("==").join(parts)); case "<>": - return String.format(" ( %s ) ", Joiner.on("!=").join(parts) ); + return String.format(" ( %s ) ", Joiner.on("!=").join(parts)); default: - return String.format(" ( %s ) ", Joiner.on(cdn.op.getName().toUpperCase()).join(parts) ); + return String.format(" ( %s ) ", Joiner.on(cdn.op.getName().toUpperCase()).join(parts)); } case "SqlCaseOperator": // CASE return String.format(" (%s ? %s : %s)", parts.get(0), parts.get(1), parts.get(2)); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java deleted file mode 100644 index bf475da97e0c..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectRule.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.beam.sdk.java.sql.interpreter; - -import java.io.Serializable; - -@Deprecated -public class ProjectRule implements Serializable { - /** - * - */ - private static final long serialVersionUID = 6166324769404546121L; - private ProjectType type; - private int sourceIndex;// for RexInputRef - - private String projectExp; - - public ProjectRule() { - } - - public ProjectType getType() { - return type; - } - - public void setType(ProjectType type) { - this.type = type; - } - - public String getProjectExp() { - return projectExp; - } - - public void setProjectExp(String projectExp) { - this.projectExp = projectExp; - } - - public int getSourceIndex() { - return sourceIndex; - } - - public void setSourceIndex(int sourceIndex) { - this.sourceIndex = sourceIndex; - } - - @Override - public String toString() { - return "ProjectRule [type=" + type + ", sourceIndex=" + sourceIndex + ", projectExp=" - + projectExp + "]"; - } - -} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java deleted file mode 100644 index abcf69c89701..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/ProjectType.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.beam.sdk.java.sql.interpreter; - -import java.io.Serializable; - -@Deprecated -public enum ProjectType implements Serializable { - RexLiteral, RexInputRef, RexCall; -} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java index 007ee431af67..6ed4366195ef 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java @@ -16,7 +16,8 @@ * limitations under the License. */ /** - * BeamSQL provides a new interface to generate Beam pipeline from a SQL statement. + * BeamSQL provides a new interface to generate Beam pipeline from a SQL + * statement. * */ package org.beam.sdk.java.sql; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java index 3d7f555cd32f..53b3f577e2cf 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java @@ -20,23 +20,16 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.io.kafka.KafkaRecord; -import org.apache.beam.sdk.io.kafka.KafkaRecordCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.beam.sdk.java.sql.rel.BeamRelNode; import org.beam.sdk.java.sql.schema.BaseBeamTable; import org.beam.sdk.java.sql.schema.BeamSQLRow; -import org.beam.sdk.java.sql.transform.BeamSQLOutputToConsoleFn; - -import io.ebay.rheos.schema.event.RheosEvent; /** - * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam pipeline. + * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam + * pipeline. * */ public class BeamPipelineCreator { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java index 941ec55b6e0e..09408fa77523 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java @@ -52,12 +52,16 @@ import org.beam.sdk.java.sql.rel.BeamLogicalConvention; import org.beam.sdk.java.sql.rel.BeamRelNode; import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * The core component to handle through a SQL statement, to submit a Beam pipeline. + * The core component to handle through a SQL statement, to submit a Beam + * pipeline. * */ public class BeamQueryPlanner { + private static final Logger LOG = LoggerFactory.getLogger(BeamQueryPlanner.class); protected final Planner planner; private Map sourceTables = new HashMap<>(); @@ -79,9 +83,7 @@ public BeamQueryPlanner(SchemaPlus schema) { Collections.emptyList(), typeFactory)); FrameworkConfig config = Frameworks.newConfigBuilder() - .parserConfig(SqlParser.configBuilder() - .setLex(Lex.MYSQL).build()) - .defaultSchema(schema) + .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema) .traitDefs(traitDefs).context(Contexts.EMPTY_CONTEXT).ruleSets(BeamRuleSets.getRuleSets()) .costFactory(null).typeSystem(BeamRelDataTypeSystem.BEAM_REL_DATATYPE_SYSTEM).build(); this.planner = Frameworks.getPlanner(config); @@ -92,20 +94,22 @@ public BeamQueryPlanner(SchemaPlus schema) { } /** - * With a Beam pipeline generated in {@link #compileBeamPipeline(String)}, submit it to run and wait until finish. + * With a Beam pipeline generated in {@link #compileBeamPipeline(String)}, + * submit it to run and wait until finish. * * @param sqlStatement * @throws Exception */ - public void submitToRun(String sqlStatement) throws Exception{ + public void submitToRun(String sqlStatement) throws Exception { Pipeline pipeline = compileBeamPipeline(sqlStatement); - + PipelineResult result = pipeline.run(); result.waitUntilFinish(); } - + /** - * With the @{@link BeamRelNode} tree generated in {@link #convertToBeamRel(String)}, a Beam pipeline is generated. + * With the @{@link BeamRelNode} tree generated in + * {@link #convertToBeamRel(String)}, a Beam pipeline is generated. * * @param sqlStatement * @return @@ -119,7 +123,9 @@ public Pipeline compileBeamPipeline(String sqlStatement) throws Exception { } /** - * It parses and validate the input query, then convert into a {@link BeamRelNode} tree. + * It parses and validate the input query, then convert into a + * {@link BeamRelNode} tree. + * * @param query * @return * @throws ValidationException @@ -134,15 +140,15 @@ public BeamRelNode convertToBeamRel(String sqlStatement) private RelNode validateAndConvert(SqlNode sqlNode) throws ValidationException, RelConversionException { SqlNode validated = validateNode(sqlNode); + LOG.info("SQL:\n" + validated); RelNode relNode = convertToRelNode(validated); return convertToBeamRel(relNode); } private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException { RelTraitSet traitSet = relNode.getTraitSet(); - // traitSet = traitSet.simplify(); - System.out.println("SQLPlan>\n" + RelOptUtil.toString(relNode)); + LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode)); // PlannerImpl.transform() optimizes RelNode with ruleset return planner.transform(0, traitSet.replace(BeamLogicalConvention.INSTANCE), relNode); @@ -162,5 +168,4 @@ public Map getSourceTables() { return sourceTables; } - } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java index a189cae414c2..892a49885498 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java @@ -32,20 +32,19 @@ import com.google.common.collect.ImmutableSet; /** - * {@link RuleSet} which translate a standard Calcite {@link RelNode} tree, to represent with {@link BeamRelNode} + * {@link RuleSet} which translate a standard Calcite {@link RelNode} tree, to + * represent with {@link BeamRelNode} * */ public class BeamRuleSets { private static final ImmutableSet calciteToBeamConversionRules = ImmutableSet - .builder() - .add( - BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE, BeamFilterRule.INSTANCE, - BeamIOSinkRule.INSTANCE - ).build(); + .builder().add(BeamIOSourceRule.INSTANCE, BeamProjectRule.INSTANCE, + BeamFilterRule.INSTANCE, BeamIOSinkRule.INSTANCE) + .build(); public static RuleSet[] getRuleSets() { - return new RuleSet[] {new BeamRuleSet(ImmutableSet.builder() - .addAll(calciteToBeamConversionRules).build()) }; + return new RuleSet[] { new BeamRuleSet( + ImmutableSet.builder().addAll(calciteToBeamConversionRules).build()) }; } private static class BeamRuleSet implements RuleSet { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java index c274e013df0c..b08c06511843 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java @@ -17,54 +17,31 @@ */ package org.beam.sdk.java.sql.planner; -import java.io.IOException; import java.io.Serializable; -import java.sql.SQLException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; import org.apache.calcite.adapter.java.JavaTypeFactory; import org.apache.calcite.jdbc.JavaTypeFactoryImpl; import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.rel.RelCollations; -import org.apache.calcite.rel.RelFieldCollation; -import org.apache.calcite.rel.RelFieldCollation.Direction; -import org.apache.calcite.rel.RelFieldCollation.NullDirection; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; -import org.apache.calcite.schema.Statistic; -import org.apache.calcite.schema.Statistics; -import org.apache.calcite.schema.Table; import org.apache.calcite.sql.parser.SqlParseException; -import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.RelConversionException; import org.apache.calcite.tools.ValidationException; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.beam.sdk.java.sql.planner.BeamQueryPlanner; import org.beam.sdk.java.sql.rel.BeamRelNode; import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import org.beam.sdk.java.sql.schema.kafka.BeamKafkaTable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.collect.ImmutableList; - public class BeamSqlRunner implements Serializable { /** * */ private static final long serialVersionUID = -4708693435115005182L; - + private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRunner.class); - + private JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); private SchemaPlus schema = Frameworks.createRootSchema(true); @@ -82,7 +59,7 @@ public void addSchema(String schemaName, Schema scheme) { /** * add a {@link BaseBeamTable} to schema repository. - * + * * @param tableName * @param table */ @@ -97,11 +74,11 @@ public void addTable(String tableName, BaseBeamTable table) { * @param sqlString * @throws Exception */ - public void submitQuery(String sqlString) throws Exception{ + public void submitQuery(String sqlString) throws Exception { planner.submitToRun(sqlString); planner.planner.close(); } - + /** * explain and display the execution plan. * @@ -110,11 +87,11 @@ public void submitQuery(String sqlString) throws Exception{ * @throws RelConversionException * @throws SqlParseException */ - public void explainQuery(String sqlString) throws ValidationException, RelConversionException, SqlParseException { + public void explainQuery(String sqlString) + throws ValidationException, RelConversionException, SqlParseException { BeamRelNode exeTree = planner.convertToBeamRel(sqlString); String beamPlan = RelOptUtil.toString(exeTree); - LOG.info("beamPlan>"); - LOG.info(beamPlan); + System.out.println(String.format("beamPlan>\n%s", beamPlan)); planner.planner.close(); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java index 652a4f91fef0..5e613aba69fd 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java @@ -16,7 +16,9 @@ * limitations under the License. */ /** - * {@link org.beam.sdk.java.sql.planner.BeamQueryPlanner} is the main interface, to define data sources, validate a SQL statement, and convert it as a Beam pipeline. + * {@link org.beam.sdk.java.sql.planner.BeamQueryPlanner} is the main interface, + * to define data sources, validate a SQL statement, and convert it as a Beam + * pipeline. * */ package org.beam.sdk.java.sql.planner; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java index e8f311f7e92d..c7e5fdf004da 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java @@ -43,7 +43,7 @@ public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, public Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition) { return new BeamFilterRel(getCluster(), traitSet, input, condition); } - + @Override public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { @@ -64,5 +64,4 @@ public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Except return planCreator.getPipeline(); } - } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java index a671d5a661d3..df345bb2179f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java @@ -20,10 +20,6 @@ import java.util.List; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.kafka.KafkaIO; -import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; @@ -49,11 +45,11 @@ public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable tabl @Override public RelNode copy(RelTraitSet traitSet, List inputs) { - return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), - sole(inputs), getOperation(), getUpdateColumnList(), getSourceExpressionList(), - isFlattened()); + return new BeamIOSinkRel(getCluster(), traitSet, getTable(), getCatalogReader(), sole(inputs), + getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened()); } + @SuppressWarnings({ "unchecked", "rawtypes" }) @Override public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { @@ -68,13 +64,12 @@ public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Except BaseBeamTable targetTable = planCreator.getSourceTables().get(sourceName); - PCollection formattedOutput = upstream.apply("preformat_to_target", targetTable.getOutputTransform()); - formattedOutput.apply("persistent", targetTable.buildIOWriter()); + PCollection preformattedStream = upstream.apply("preformat_to_target", targetTable.getOutputTransform()); + preformattedStream.apply("persistent", targetTable.buildIOWriter()); planCreator.setHasPersistent(true); return planCreator.getPipeline(); } - } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java index 760e78987b6f..e1d822213f1d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java @@ -36,6 +36,7 @@ public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable super(cluster, traitSet, table); } + @SuppressWarnings({ "unchecked", "rawtypes" }) @Override public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { @@ -45,9 +46,10 @@ public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Except String stageName = BeamSQLRelUtils.getStageName(this); - PCollection sourceStream = planCreator.getPipeline() - .apply(stageName, sourceTable.buildIOReader()); - PCollection reformattedSourceStream = sourceStream.apply("sourceReformat", sourceTable.getInputTransform()); + PCollection sourceStream = planCreator.getPipeline().apply(stageName, + sourceTable.buildIOReader()); + PCollection reformattedSourceStream = sourceStream.apply("sourceReformat", + sourceTable.getInputTransform()); planCreator.setLatestStream(reformattedSourceStream); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java index 34ce54da91bc..06e7331b228d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java @@ -71,14 +71,14 @@ public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Except PCollection upstream = planCreator.getLatestStream(); BeamSQLExpressionExecutor executor = new BeamSQLSpELExecutor(this); - - PCollection projectStream = upstream.apply(stageName, - ParDo.of(new BeamSQLProjectFn(getRelTypeName(), executor, BeamSQLRecordType.from(rowType)))); + + PCollection projectStream = upstream.apply(stageName, ParDo + .of(new BeamSQLProjectFn(getRelTypeName(), executor, BeamSQLRecordType.from(rowType)))); planCreator.setLatestStream(projectStream); return planCreator.getPipeline(); - + } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java index 78ebf03c9d17..94ab20e1eb01 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java @@ -22,13 +22,17 @@ import org.beam.sdk.java.sql.planner.BeamPipelineCreator; /** - * A new method {@link #buildBeamPipeline(BeamPipelineCreator)} is added, it's called by {@link BeamPipelineCreator}. + * A new method {@link #buildBeamPipeline(BeamPipelineCreator)} is added, it's + * called by {@link BeamPipelineCreator}. * */ public interface BeamRelNode extends RelNode { - + /** - * A {@link BeamRelNode} is a recursive structure, the {@link BeamPipelineCreator} visits it with a DFS(Depth-First-Search) algorithm. + * A {@link BeamRelNode} is a recursive structure, the + * {@link BeamPipelineCreator} visits it with a DFS(Depth-First-Search) + * algorithm. + * * @param planCreator * @throws Exception */ diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java index 8daf8dacf74b..4f058bba41ce 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java @@ -29,8 +29,7 @@ public class BeamProjectRule extends ConverterRule { public static final BeamProjectRule INSTANCE = new BeamProjectRule(); private BeamProjectRule() { - super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, - "BeamProjectRule"); + super(LogicalProject.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamProjectRule"); } @Override diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java index fcc5ba55e2ea..1a87fba7fd87 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java @@ -43,9 +43,11 @@ public abstract class BaseBeamTable implements ScannableTable, Serializable { */ private static final long serialVersionUID = -1262988061830914193L; private RelProtoDataType protoRowType; - // A {@link PTransform} that convert a input record of {@code IO.read()} to {@link BeamSQLRow} + // A {@link PTransform} that convert a input record of {@code IO.read()} to + // {@link BeamSQLRow} private PTransform, PCollection> inputTransform; - // A {@link PTransform} that convert a {@link BeamSQLRow} to the required record of {@code IO.write()} + // A {@link PTransform} that convert a {@link BeamSQLRow} to the required + // record of {@code IO.write()} private PTransform, PCollection> outputTransform; public BaseBeamTable(RelProtoDataType protoRowType, @@ -76,10 +78,10 @@ public BaseBeamTable(RelProtoDataType protoRowType, */ public abstract PTransform, PDone> buildIOWriter(); - - /** - * A {@link PTransform} that convert a input record of {@code IO.read()} to {@link BeamSQLRow} + * A {@link PTransform} that convert a input record of {@code IO.read()} to + * {@link BeamSQLRow} + * * @return */ public PTransform, PCollection> getInputTransform() { @@ -87,7 +89,9 @@ public PTransform, PCollection> getInputTransform() { } /** - * A {@link PTransform} that convert a {@link BeamSQLRow} to the required record of {@code IO.write()} + * A {@link PTransform} that convert a {@link BeamSQLRow} to the required + * record of {@code IO.write()} + * * @return */ public PTransform, PCollection> getOutputTransform() { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java index 30010834460d..3aa12b5d7c8a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java @@ -53,8 +53,8 @@ public void addField(String fieldName, Object fieldValue) { // dataMap.put(fieldName, null); } } - - public void addField(int index, Object fieldValue){ + + public void addField(int index, Object fieldValue) { addField(dataType.getFieldsName().get(index), fieldValue); } @@ -80,7 +80,7 @@ private Object getFieldValue(String fieldName, String fieldType) { return Integer.valueOf(dataMap.get(fieldName)); case VARCHAR: return dataMap.get(fieldName); - case TIMESTAMP: //TODO + case TIMESTAMP: // TODO case BIGINT: return Long.valueOf(dataMap.get(fieldName)); default: @@ -127,9 +127,7 @@ public boolean equals(Object obj) { if (getClass() != obj.getClass()) return false; BeamSQLRow other = (BeamSQLRow) obj; - return toString().equals(other.toString()); + return toString().equals(other.toString()); } - } - diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index 846e156a8301..b892d7ee6466 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -25,17 +25,12 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.io.kafka.KafkaIO; -import org.apache.beam.sdk.io.kafka.KafkaRecord; -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.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.calcite.rel.type.RelProtoDataType; -import org.beam.sdk.java.sql.examples.RheosSinkTransform; -import org.beam.sdk.java.sql.examples.RheosSourceTransform; import org.beam.sdk.java.sql.schema.BaseBeamTable; import org.beam.sdk.java.sql.schema.BeamIOType; import org.beam.sdk.java.sql.schema.BeamSQLRow; @@ -80,8 +75,7 @@ public BeamIOType getSourceType() { public PTransform>> buildIOReader() { return KafkaIO.read().withBootstrapServers(this.bootstrapServers) .withTopics(this.topics).updateConsumerProperties(configUpdates) - .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()) - .withoutMetadata(); + .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()).withoutMetadata(); } @Override @@ -90,8 +84,8 @@ public PTransform>, PDone> buildIOWriter( "Only one topic can be acceptable as output."); return KafkaIO.write().withBootstrapServers(bootstrapServers) - .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) - .withValueCoder(ByteArrayCoder.of()); + .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) + .withValueCoder(ByteArrayCoder.of()); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java deleted file mode 100644 index 321fa748b32b..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/KafkaCSVRecordTransform.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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.beam.sdk.java.sql.schema.kafka; - -import org.apache.beam.sdk.io.kafka.KafkaRecord; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PCollection; -import org.beam.sdk.java.sql.schema.BeamSQLRow; - -public class KafkaCSVRecordTransform { - - - public static class ReaderTransform extends PTransform>, PCollection>{ - /** - * - */ - private static final long serialVersionUID = 7613394830984433222L; - - @Override - public PCollection expand(PCollection> input) { - // TODO Auto-generated method stub - return null; - } - - } - - public static class SinkerTransform extends PTransform, PCollection>>{ - - /** - * - */ - private static final long serialVersionUID = -722396312765710736L; - - @Override - public PCollection> expand(PCollection input) { - // TODO Auto-generated method stub - return null; - } - - } - -} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java index 60aa1d644451..803f14ee3ce4 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java @@ -49,13 +49,13 @@ public void processElement(ProcessContext c) { List result = executor.execute(in); - if ((Boolean)result.get(0)) { + if ((Boolean) result.get(0)) { c.output(in); } } - + @Teardown - public void close(){ + public void close() { executor.close(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java index a2d0a93724a6..d096e807c12a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java @@ -34,7 +34,8 @@ public class BeamSQLProjectFn extends DoFn { private BeamSQLExpressionExecutor executor; private BeamSQLRecordType outputRecordType; - public BeamSQLProjectFn(String stepName, BeamSQLExpressionExecutor executor, BeamSQLRecordType outputRecordType) { + public BeamSQLProjectFn(String stepName, BeamSQLExpressionExecutor executor, + BeamSQLRecordType outputRecordType) { super(); this.stepName = stepName; this.executor = executor; @@ -49,17 +50,17 @@ public void setup() { @ProcessElement public void processElement(ProcessContext c) { List results = executor.execute(c.element()); - + BeamSQLRow outRow = new BeamSQLRow(outputRecordType); - for(int idx=0; idx Date: Sat, 11 Mar 2017 11:15:49 -0800 Subject: [PATCH 09/15] fix checkstyle 1 --- .../sdk/java/sql/examples/package-info.java | 22 -------- .../BeamSQLExpressionExecutor.java | 2 +- .../sql/interpreter/BeamSQLSpELExecutor.java | 2 +- .../java/sql/interpreter/CalciteToSpEL.java | 5 +- .../java/sql/interpreter/package-info.java | 4 +- .../org/beam/sdk/java/sql/package-info.java | 6 +-- .../java/sql/planner/BeamPipelineCreator.java | 2 +- .../java/sql/planner/BeamQueryPlanner.java | 8 +-- .../sdk/java/sql/planner/BeamRuleSets.java | 11 ++-- .../sdk/java/sql/planner/BeamSQLRelUtils.java | 9 +++- .../sdk/java/sql/planner/BeamSqlRunner.java | 14 +++-- .../planner/BeamSqlUnsupportedException.java | 6 ++- .../planner/UnsupportedOperatorsVisitor.java | 4 ++ .../sdk/java/sql/planner/package-info.java | 8 +-- .../beam/sdk/java/sql/rel/BeamFilterRel.java | 4 ++ .../beam/sdk/java/sql/rel/BeamIOSinkRel.java | 12 +++-- .../sdk/java/sql/rel/BeamIOSourceRel.java | 13 ++--- .../java/sql/rel/BeamLogicalConvention.java | 4 ++ .../beam/sdk/java/sql/rel/BeamProjectRel.java | 8 ++- .../beam/sdk/java/sql/rel/BeamRelNode.java | 2 +- .../beam/sdk/java/sql/rel/package-info.java | 5 +- .../sdk/java/sql/rule/BeamFilterRule.java | 4 ++ .../sdk/java/sql/rule/BeamIOSinkRule.java | 4 ++ .../sdk/java/sql/rule/BeamIOSourceRule.java | 4 ++ .../sdk/java/sql/rule/BeamProjectRule.java | 4 ++ .../beam/sdk/java/sql/rule/package-info.java | 6 +-- .../sdk/java/sql/schema/BaseBeamTable.java | 53 ++++++------------- .../java/sql/schema/BeamSQLRecordType.java | 6 ++- .../beam/sdk/java/sql/schema/BeamSQLRow.java | 15 ++++-- .../java/sql/schema/kafka/BeamKafkaTable.java | 46 ++++++++-------- .../java/sql/schema/kafka/package-info.java | 6 +-- .../sdk/java/sql/schema/package-info.java | 5 +- .../java/sql/transform/BeamSQLFilterFn.java | 7 ++- .../transform/BeamSQLOutputToConsoleFn.java | 6 ++- .../java/sql/transform/BeamSQLProjectFn.java | 8 ++- .../sdk/java/sql/transform/package-info.java | 6 +-- .../beam/sdk/java/sql/e2e/package-info.java | 6 +-- 37 files changed, 186 insertions(+), 151 deletions(-) delete mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java deleted file mode 100644 index f42eaaf55597..000000000000 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/examples/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * 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. - */ -/** - * Contains some examples on how to use BeamSQL to simplify pipeline assembling. - * - */ -package org.beam.sdk.java.sql.examples; \ No newline at end of file diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java index 24785569da21..6d341e9ea06d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java @@ -36,7 +36,7 @@ public interface BeamSQLExpressionExecutor extends Serializable { /** * apply transformation to input record {@link BeamSQLRow}. - * + * * @param inputRecord * @return */ diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java index 6578fb912626..c1fbb85eacc1 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java @@ -44,7 +44,7 @@ */ public class BeamSQLSpELExecutor implements BeamSQLExpressionExecutor { /** - * + * */ private static final long serialVersionUID = 6777232573390074408L; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java index aac23cbe734a..9b7c46c9d93f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java @@ -17,16 +17,17 @@ */ package org.beam.sdk.java.sql.interpreter; +import com.google.common.base.Joiner; + import java.util.ArrayList; import java.util.List; + import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.beam.sdk.java.sql.planner.BeamSqlUnsupportedException; -import com.google.common.base.Joiner; - /** * {@code CalciteToSpEL} is used in {@link BeamSQLSpELExecutor}, to convert a * relational expression {@link RexCall} to SpEL expression. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java index 43d4651c94a2..997e16a182b3 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java @@ -15,8 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** * interpreter generate runnable 'code' to execute SQL relational expressions. - * */ -package org.beam.sdk.java.sql.interpreter; \ No newline at end of file +package org.beam.sdk.java.sql.interpreter; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java index 6ed4366195ef..847c1a4920f0 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * BeamSQL provides a new interface to generate Beam pipeline from a SQL - * statement. - * + * BeamSQL provides a new interface to run a SQL statement with Beam. */ package org.beam.sdk.java.sql; + diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java index 53b3f577e2cf..71df7b602c5e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java @@ -30,7 +30,7 @@ /** * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam * pipeline. - * + * */ public class BeamPipelineCreator { private Map sourceTables; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java index 09408fa77523..d99b7cd0c0ac 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java @@ -69,7 +69,7 @@ public class BeamQueryPlanner { private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); /** - * + * * @param schema */ public BeamQueryPlanner(SchemaPlus schema) { @@ -96,7 +96,7 @@ public BeamQueryPlanner(SchemaPlus schema) { /** * With a Beam pipeline generated in {@link #compileBeamPipeline(String)}, * submit it to run and wait until finish. - * + * * @param sqlStatement * @throws Exception */ @@ -110,7 +110,7 @@ public void submitToRun(String sqlStatement) throws Exception { /** * With the @{@link BeamRelNode} tree generated in * {@link #convertToBeamRel(String)}, a Beam pipeline is generated. - * + * * @param sqlStatement * @return * @throws Exception @@ -125,7 +125,7 @@ public Pipeline compileBeamPipeline(String sqlStatement) throws Exception { /** * It parses and validate the input query, then convert into a * {@link BeamRelNode} tree. - * + * * @param query * @return * @throws ValidationException diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java index 892a49885498..814807e37d46 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java @@ -17,6 +17,9 @@ */ package org.beam.sdk.java.sql.planner; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + import java.util.Iterator; import org.apache.calcite.plan.RelOptRule; @@ -28,13 +31,11 @@ import org.beam.sdk.java.sql.rule.BeamIOSourceRule; import org.beam.sdk.java.sql.rule.BeamProjectRule; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; - /** - * {@link RuleSet} which translate a standard Calcite {@link RelNode} tree, to + * {@link RuleSet} used in {@link BeamQueryPlanner}. + * It translates a standard Calcite {@link RelNode} tree, to * represent with {@link BeamRelNode} - * + * */ public class BeamRuleSets { private static final ImmutableSet calciteToBeamConversionRules = ImmutableSet diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java index 88e028e2f941..4f4019046b68 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java @@ -27,6 +27,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Utilities for {@code BeamRelNode}. + */ public class BeamSQLRelUtils { private static final Logger LOG = LoggerFactory.getLogger(BeamSQLRelUtils.class); @@ -61,10 +64,12 @@ public static String explain(final RelNode rel, SqlExplainLevel detailLevel) { explain = RelOptUtil.toString(rel); } catch (StackOverflowError e) { LOG.error( - "StackOverflowError occurred while extracting plan. Please report it to the dev@ mailing list."); + "StackOverflowError occurred while extracting plan. " + + "Please report it to the dev@ mailing list."); LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e); LOG.error( - "Forcing plan to empty string and continue... SQL Runner may not working properly after."); + "Forcing plan to empty string and continue... " + + "SQL Runner may not working properly after."); } return explain; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java index b08c06511843..0c804d972041 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java @@ -34,9 +34,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * Interface to explain, submit a SQL query. + * + */ public class BeamSqlRunner implements Serializable { /** - * + * */ private static final long serialVersionUID = -4708693435115005182L; @@ -49,7 +53,7 @@ public class BeamSqlRunner implements Serializable { /** * Add a schema. - * + * * @param schemaName * @param scheme */ @@ -59,7 +63,7 @@ public void addSchema(String schemaName, Schema scheme) { /** * add a {@link BaseBeamTable} to schema repository. - * + * * @param tableName * @param table */ @@ -70,7 +74,7 @@ public void addTable(String tableName, BaseBeamTable table) { /** * submit as a Beam pipeline. - * + * * @param sqlString * @throws Exception */ @@ -81,7 +85,7 @@ public void submitQuery(String sqlString) throws Exception { /** * explain and display the execution plan. - * + * * @param sqlString * @throws ValidationException * @throws RelConversionException diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java index 49ccb12dce5f..1aa2c7ceefd2 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java @@ -17,9 +17,13 @@ */ package org.beam.sdk.java.sql.planner; +/** + * Generic exception for un-supported operations. + * + */ public class BeamSqlUnsupportedException extends RuntimeException { /** - * + * */ private static final long serialVersionUID = 3445015747629217342L; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java index daf720837034..c95804395d29 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java @@ -19,6 +19,10 @@ import org.apache.calcite.sql.util.SqlShuttle; +/** + * Unsupported operation to visit a RelNode. + * + */ public class UnsupportedOperatorsVisitor extends SqlShuttle { } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java index 5e613aba69fd..b62bf67aed51 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * {@link org.beam.sdk.java.sql.planner.BeamQueryPlanner} is the main interface, - * to define data sources, validate a SQL statement, and convert it as a Beam + * {@link org.beam.sdk.java.sql.planner.BeamQueryPlanner} is the main interface. + * It defines data sources, validate a SQL statement, and convert it as a Beam * pipeline. - * */ -package org.beam.sdk.java.sql.planner; \ No newline at end of file +package org.beam.sdk.java.sql.planner; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java index c7e5fdf004da..c5cac6d05f87 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java @@ -32,6 +32,10 @@ import org.beam.sdk.java.sql.schema.BeamSQLRow; import org.beam.sdk.java.sql.transform.BeamSQLFilterFn; +/** + * BeamRelNode to replace a {@code Filter} node. + * + */ public class BeamFilterRel extends Filter implements BeamRelNode { public BeamFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java index df345bb2179f..452cf7c5f8dc 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java @@ -17,6 +17,8 @@ */ package org.beam.sdk.java.sql.rel; +import com.google.common.base.Joiner; + import java.util.List; import org.apache.beam.sdk.Pipeline; @@ -33,8 +35,10 @@ import org.beam.sdk.java.sql.schema.BaseBeamTable; import org.beam.sdk.java.sql.schema.BeamSQLRow; -import com.google.common.base.Joiner; - +/** + * BeamRelNode to replace a {@code TableModify} node. + * + */ public class BeamIOSinkRel extends TableModify implements BeamRelNode { public BeamIOSinkRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, Prepare.CatalogReader catalogReader, RelNode child, Operation operation, @@ -49,7 +53,6 @@ public RelNode copy(RelTraitSet traitSet, List inputs) { getOperation(), getUpdateColumnList(), getSourceExpressionList(), isFlattened()); } - @SuppressWarnings({ "unchecked", "rawtypes" }) @Override public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { @@ -64,8 +67,7 @@ public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Except BaseBeamTable targetTable = planCreator.getSourceTables().get(sourceName); - PCollection preformattedStream = upstream.apply("preformat_to_target", targetTable.getOutputTransform()); - preformattedStream.apply("persistent", targetTable.buildIOWriter()); + upstream.apply(stageName, targetTable.buildIOWriter()); planCreator.setHasPersistent(true); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java index e1d822213f1d..f9ac35cdda7c 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java @@ -17,6 +17,8 @@ */ package org.beam.sdk.java.sql.rel; +import com.google.common.base.Joiner; + import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; @@ -28,15 +30,16 @@ import org.beam.sdk.java.sql.schema.BaseBeamTable; import org.beam.sdk.java.sql.schema.BeamSQLRow; -import com.google.common.base.Joiner; - +/** + * BeamRelNode to replace a {@code TableScan} node. + * + */ public class BeamIOSourceRel extends TableScan implements BeamRelNode { public BeamIOSourceRel(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) { super(cluster, traitSet, table); } - @SuppressWarnings({ "unchecked", "rawtypes" }) @Override public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Exception { @@ -48,10 +51,8 @@ public Pipeline buildBeamPipeline(BeamPipelineCreator planCreator) throws Except PCollection sourceStream = planCreator.getPipeline().apply(stageName, sourceTable.buildIOReader()); - PCollection reformattedSourceStream = sourceStream.apply("sourceReformat", - sourceTable.getInputTransform()); - planCreator.setLatestStream(reformattedSourceStream); + planCreator.setLatestStream(sourceStream); return planCreator.getPipeline(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java index 18d7eaf6bb38..739bc012189b 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java @@ -24,6 +24,10 @@ import org.apache.calcite.plan.RelTraitDef; import org.apache.calcite.plan.RelTraitSet; +/** + * Convertion for Beam SQL. + * + */ public enum BeamLogicalConvention implements Convention { INSTANCE; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java index 06e7331b228d..9086c6c0f593 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java @@ -39,11 +39,15 @@ import org.beam.sdk.java.sql.schema.BeamSQLRow; import org.beam.sdk.java.sql.transform.BeamSQLProjectFn; +/** + * BeamRelNode to replace a {@code Project} node. + * + */ public class BeamProjectRel extends Project implements BeamRelNode { /** - * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall} - * + * projects: {@link RexLiteral}, {@link RexInputRef}, {@link RexCall}. + * * @param cluster * @param traits * @param input diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java index 94ab20e1eb01..65bab08c7855 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java @@ -32,7 +32,7 @@ public interface BeamRelNode extends RelNode { * A {@link BeamRelNode} is a recursive structure, the * {@link BeamPipelineCreator} visits it with a DFS(Depth-First-Search) * algorithm. - * + * * @param planCreator * @throws Exception */ diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java index 8e0df21e5e7a..bd3cf1d96151 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java @@ -15,8 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode} + * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}. * */ -package org.beam.sdk.java.sql.rel; \ No newline at end of file +package org.beam.sdk.java.sql.rel; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java index ee8185a9de1e..959996a270ab 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java @@ -25,6 +25,10 @@ import org.beam.sdk.java.sql.rel.BeamFilterRel; import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +/** + * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}. + * + */ public class BeamFilterRule extends ConverterRule { public static BeamFilterRule INSTANCE = new BeamFilterRule(); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java index b4eb1daa4be6..73271c393cec 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java @@ -33,6 +33,10 @@ import org.beam.sdk.java.sql.rel.BeamIOSinkRel; import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +/** + * A {@code ConverterRule} to replace {@link TableModify} with {@link BeamIOSinkRel}. + * + */ public class BeamIOSinkRule extends ConverterRule { public static final BeamIOSinkRule INSTANCE = new BeamIOSinkRule(); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java index 5e62e8021658..aa364aea256b 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java @@ -25,6 +25,10 @@ import org.beam.sdk.java.sql.rel.BeamIOSourceRel; import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +/** + * A {@code ConverterRule} to replace {@link TableScan} with {@link BeamIOSourceRel}. + * + */ public class BeamIOSourceRule extends ConverterRule { public static final BeamIOSourceRule INSTANCE = new BeamIOSourceRule(); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java index 4f058bba41ce..3eee3b29a8e5 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java @@ -25,6 +25,10 @@ import org.beam.sdk.java.sql.rel.BeamLogicalConvention; import org.beam.sdk.java.sql.rel.BeamProjectRel; +/** + * A {@code ConverterRule} to replace {@link Project} with {@link BeamProjectRel}. + * + */ public class BeamProjectRule extends ConverterRule { public static final BeamProjectRule INSTANCE = new BeamProjectRule(); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java index 502bdc70cfab..d363949c8933 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java @@ -15,8 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * - * + * {@link RelOptRule} to generate {@link BeamRelNode}. */ -package org.beam.sdk.java.sql.rule; \ No newline at end of file +package org.beam.sdk.java.sql.rule; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java index 1a87fba7fd87..e388cf262ff1 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java @@ -36,26 +36,23 @@ /** * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}. */ -public abstract class BaseBeamTable implements ScannableTable, Serializable { +public abstract class BaseBeamTable implements ScannableTable, Serializable { /** - * + * */ private static final long serialVersionUID = -1262988061830914193L; private RelProtoDataType protoRowType; - // A {@link PTransform} that convert a input record of {@code IO.read()} to - // {@link BeamSQLRow} - private PTransform, PCollection> inputTransform; - // A {@link PTransform} that convert a {@link BeamSQLRow} to the required - // record of {@code IO.write()} - private PTransform, PCollection> outputTransform; + // // A {@link PTransform} that convert a input record of {@code IO.read()} to + // // {@link BeamSQLRow} + // private PTransform, PCollection> inputTransform; + // // A {@link PTransform} that convert a {@link BeamSQLRow} to the required + // // record of {@code IO.write()} + // private PTransform, PCollection> + // outputTransform; - public BaseBeamTable(RelProtoDataType protoRowType, - PTransform, PCollection> inputTransform, - PTransform, PCollection> outputTransform) { + public BaseBeamTable(RelProtoDataType protoRowType) { this.protoRowType = protoRowType; - this.inputTransform = inputTransform; - this.outputTransform = outputTransform; } /** @@ -66,37 +63,17 @@ public BaseBeamTable(RelProtoDataType protoRowType, /** * create a {@code IO.read()} instance to read from source. - * + * * @return */ - public abstract PTransform> buildIOReader(); + public abstract PTransform> buildIOReader(); /** * create a {@code IO.write()} instance to write to target. - * + * * @return */ - public abstract PTransform, PDone> buildIOWriter(); - - /** - * A {@link PTransform} that convert a input record of {@code IO.read()} to - * {@link BeamSQLRow} - * - * @return - */ - public PTransform, PCollection> getInputTransform() { - return inputTransform; - } - - /** - * A {@link PTransform} that convert a {@link BeamSQLRow} to the required - * record of {@code IO.write()} - * - * @return - */ - public PTransform, PCollection> getOutputTransform() { - return outputTransform; - } + public abstract PTransform, PDone> buildIOWriter(); @Override public Enumerable scan(DataContext root) { @@ -110,7 +87,7 @@ public RelDataType getRowType(RelDataTypeFactory typeFactory) { } /** - * Not used {@link Statistic} to optimize the plan + * Not used {@link Statistic} to optimize the plan. */ @Override public Statistic getStatistic() { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java index c3dffbc7948d..36734e92498a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java @@ -26,10 +26,14 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; +/** + * Field type information in {@link BeamSQLRow}. + * + */ @DefaultCoder(AvroCoder.class) public class BeamSQLRecordType implements Serializable { /** - * + * */ private static final long serialVersionUID = -5318734648766104712L; private List fieldsName = new ArrayList<>(); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java index 3aa12b5d7c8a..2fb21b1b445d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java @@ -26,10 +26,14 @@ import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.calcite.sql.type.SqlTypeName; +/** + * Repersent a generic ROW record in Beam SQL. + * + */ @DefaultCoder(AvroCoder.class) public class BeamSQLRow implements Serializable { /** - * + * */ private static final long serialVersionUID = 4569220242480160895L; @@ -120,12 +124,15 @@ public int hashCode() { @Override public boolean equals(Object obj) { - if (this == obj) + if (this == obj) { return true; - if (obj == null) + } + if (obj == null) { return false; - if (getClass() != obj.getClass()) + } + if (getClass() != obj.getClass()) { return false; + } BeamSQLRow other = (BeamSQLRow) obj; return toString().equals(other.toString()); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index b892d7ee6466..ffa8b3aeaaac 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -23,10 +23,7 @@ import java.util.List; import java.util.Map; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -35,10 +32,15 @@ import org.beam.sdk.java.sql.schema.BeamIOType; import org.beam.sdk.java.sql.schema.BeamSQLRow; -public class BeamKafkaTable extends BaseBeamTable> implements Serializable { +/** + * {@code BeamKafkaTable} represent a Kafka topic, as source or target. + * A user PTransform is required to handle the input/output as KV. + * + */ +public class BeamKafkaTable extends BaseBeamTable implements Serializable { /** - * + * */ private static final long serialVersionUID = -634715473399906527L; @@ -46,17 +48,13 @@ public class BeamKafkaTable extends BaseBeamTable> implements private List topics; private Map configUpdates; - protected BeamKafkaTable(RelProtoDataType protoRowType, - PTransform>, PCollection> sourceConverter, - PTransform, PCollection>> sinkConcerter) { - super(protoRowType, sourceConverter, sinkConcerter); + protected BeamKafkaTable(RelProtoDataType protoRowType) { + super(protoRowType); } - public BeamKafkaTable(RelProtoDataType protoRowType, - PTransform>, PCollection> sourceConverter, - PTransform, PCollection>> sinkConcerter, - String bootstrapServers, List topics) { - super(protoRowType, sourceConverter, sinkConcerter); + public BeamKafkaTable(RelProtoDataType protoRowType, String bootstrapServers, + List topics) { + super(protoRowType); this.bootstrapServers = bootstrapServers; this.topics = topics; } @@ -72,20 +70,24 @@ public BeamIOType getSourceType() { } @Override - public PTransform>> buildIOReader() { - return KafkaIO.read().withBootstrapServers(this.bootstrapServers) - .withTopics(this.topics).updateConsumerProperties(configUpdates) - .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()).withoutMetadata(); + public PTransform> buildIOReader() { + return null; + // return KafkaIO.read().withBootstrapServers(this.bootstrapServers) + // .withTopics(this.topics).updateConsumerProperties(configUpdates) + // .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()).withoutMetadata(); } @Override - public PTransform>, PDone> buildIOWriter() { + public PTransform, PDone> buildIOWriter() { checkArgument(topics != null && topics.size() == 1, "Only one topic can be acceptable as output."); - return KafkaIO.write().withBootstrapServers(bootstrapServers) - .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) - .withValueCoder(ByteArrayCoder.of()); + return null; + // return KafkaIO.write().withBootstrapServers(bootstrapServers) + // .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) + // .withValueCoder(ByteArrayCoder.of()); } } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java index 0c0ce350417a..737fd20fe258 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java @@ -15,8 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * table schema for KafkaIO - * + * table schema for KafkaIO. */ -package org.beam.sdk.java.sql.schema.kafka; \ No newline at end of file +package org.beam.sdk.java.sql.schema.kafka; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java index 350ae3677885..046a51269e2a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java @@ -15,8 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * define table schema, to map with Beam IO components + * define table schema, to map with Beam IO components. * */ -package org.beam.sdk.java.sql.schema; \ No newline at end of file +package org.beam.sdk.java.sql.schema; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java index 803f14ee3ce4..e46c0515f916 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java @@ -21,11 +21,16 @@ import org.apache.beam.sdk.transforms.DoFn; import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.sdk.java.sql.rel.BeamFilterRel; import org.beam.sdk.java.sql.schema.BeamSQLRow; +/** + * {@code BeamSQLFilterFn} is the executor for a {@link BeamFilterRel} step. + * + */ public class BeamSQLFilterFn extends DoFn { /** - * + * */ private static final long serialVersionUID = -1256111753670606705L; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java index c78fbe2b1209..2748424fd1ea 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java @@ -20,9 +20,13 @@ import org.apache.beam.sdk.transforms.DoFn; import org.beam.sdk.java.sql.schema.BeamSQLRow; +/** + * A test PTransform to display output in console. + * + */ public class BeamSQLOutputToConsoleFn extends DoFn { /** - * + * */ private static final long serialVersionUID = -1256111753670606705L; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java index d096e807c12a..92fc66f8fab2 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java @@ -21,13 +21,19 @@ import org.apache.beam.sdk.transforms.DoFn; import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.sdk.java.sql.rel.BeamProjectRel; import org.beam.sdk.java.sql.schema.BeamSQLRecordType; import org.beam.sdk.java.sql.schema.BeamSQLRow; +/** + * + * {@code BeamSQLProjectFn} is the executor for a {@link BeamProjectRel} step. + * + */ public class BeamSQLProjectFn extends DoFn { /** - * + * */ private static final long serialVersionUID = -1046605249999014608L; private String stepName; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java index 8548a4a716b3..ddd4d1667f29 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java @@ -15,8 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * - * + * {@link PTransform} used in a BeamSQL pipeline. */ -package org.beam.sdk.java.sql.transform; \ No newline at end of file +package org.beam.sdk.java.sql.transform; diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java index 5e20fb3d6dad..fd380548578a 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java @@ -15,8 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + /** - * contain end-to-end tests - * + * contain end-to-end tests. */ -package org.beam.sdk.java.sql.e2e; \ No newline at end of file +package org.beam.sdk.java.sql.e2e; From f1afcfe7fe81ac1427a18d367c0536933cf85e63 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Sat, 11 Mar 2017 23:30:27 -0800 Subject: [PATCH 10/15] refine BaseBeamTable, and add one example BeamSqlExample. --- sdks/java/sql/pom.xml | 40 ++---- .../sdk/java/sql/example/BeamSqlExample.java | 103 ++++++++++++++ .../sdk/java/sql/example/package-info.java | 23 ++++ .../BeamSQLExpressionExecutor.java | 6 +- .../java/sql/interpreter/CalciteToSpEL.java | 1 - .../org/beam/sdk/java/sql/package-info.java | 1 - .../java/sql/planner/BeamQueryPlanner.java | 7 +- .../sdk/java/sql/planner/BeamRuleSets.java | 5 +- .../sdk/java/sql/planner/BeamSQLRelUtils.java | 6 +- .../sdk/java/sql/planner/BeamSqlRunner.java | 4 - .../sdk/java/sql/rule/BeamFilterRule.java | 2 +- .../sdk/java/sql/rule/BeamIOSinkRule.java | 5 +- .../sdk/java/sql/rule/BeamIOSourceRule.java | 3 +- .../sdk/java/sql/rule/BeamProjectRule.java | 3 +- .../sdk/java/sql/schema/BaseBeamTable.java | 17 +-- .../java/sql/schema/BeamSQLRecordType.java | 4 + .../beam/sdk/java/sql/schema/BeamSQLRow.java | 6 + .../sql/schema/kafka/BeamKafkaCSVTable.java | 128 ++++++++++++++++++ .../java/sql/schema/kafka/BeamKafkaTable.java | 58 ++++++-- .../sdk/java/sql/planner/BasePlanner.java | 41 ++++++ .../sql/planner/BeamPlannerExplainTest.java | 16 +++ 21 files changed, 403 insertions(+), 76 deletions(-) create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/BeamSqlExample.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/package-info.java create mode 100644 sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaCSVTable.java create mode 100644 sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java create mode 100644 sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java diff --git a/sdks/java/sql/pom.xml b/sdks/java/sql/pom.xml index d322228aae2f..1eb3f13da29e 100644 --- a/sdks/java/sql/pom.xml +++ b/sdks/java/sql/pom.xml @@ -19,35 +19,6 @@ yyyy-MM-dd HH:mm 1.11.0 - - - - - apache-snapshot - https://repository.apache.org/content/repositories/snapshots/ - - - apache-release - https://repository.apache.org/content/repositories/releases/ - - - ebaycentral-release - ebaycentral-release repository - http://ebaycentral/content/repositories/releases/ - - - raptor-snapshots - http://ebaycentral/content/repositories/snapshots/ - - - central - http://ebaycentral/content/repositories/central - - - raptor-thirdparty - http://ebaycentral/content/repositories/thirdparty - - @@ -180,10 +151,12 @@ org.apache.beam beam-runners-direct-java + provided org.apache.beam beam-sdks-java-io-kafka + provided org.springframework @@ -201,6 +174,15 @@ org.slf4j slf4j-api + + org.apache.avro + avro + + + org.apache.calcite + calcite-linq4j + ${calcite-version} + diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/BeamSqlExample.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/BeamSqlExample.java new file mode 100644 index 000000000000..df1032faa168 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/BeamSqlExample.java @@ -0,0 +1,103 @@ +/* + * 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.beam.sdk.java.sql.example; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.beam.sdk.java.sql.planner.BeamSqlRunner; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.kafka.BeamKafkaCSVTable; + +/** + * This is one quick example.
+ * Before start, follow https://kafka.apache.org/quickstart to setup a Kafka + * cluster locally, and run below commands to create required Kafka topics: + *
+ * 
+ * bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 \
+ *   --partitions 1 --topic orders
+ * bin/kafka-topics.sh --create --zookeeper localhost:2181 --replication-factor 1 \
+ *   --partitions 1 --topic sub_orders
+ * 
+ * 
+ * After run the application, produce several test records: + *
+ * 
+ * bin/kafka-console-producer.sh --broker-list localhost:9092 --topic orders
+ * invalid,record
+ * 123445,0,100,3413423
+ * 234123,3,232,3451231234
+ * 234234,0,5,1234123
+ * 345234,0,345234.345,3423
+ * 
+ * 
+ * Meanwhile, open another console to see the output: + *
+ * 
+ * bin/kafka-console-consumer.sh --bootstrap-server localhost:9092 --topic sub_orders
+ * **Expected :
+ * 123445,0,100.0
+ * 345234,0,345234.345
+ * 
+ * 
+ */ +public class BeamSqlExample implements Serializable { + + /** + * + */ + private static final long serialVersionUID = 3673487843555563904L; + + public static void main(String[] args) throws Exception { + BeamSqlRunner runner = new BeamSqlRunner(); + runner.addTable("ORDER_DETAILS", getTable("127.0.0.1:9092", "orders")); + runner.addTable("SUB_ORDER", getTable("127.0.0.1:9092", "sub_orders")); + + // case 2: insert into
() select STREAM from + //
from + String sql = "INSERT INTO SUB_ORDER(order_id, site_id, price) " + "SELECT " + + " order_id, site_id, price " + "FROM ORDER_DETAILS " + "WHERE SITE_ID = 0 and price > 20"; + + runner.explainQuery(sql); + runner.submitQuery(sql); + } + + public static BaseBeamTable getTable(String bootstrapServer, String topic) { + final RelProtoDataType protoRowType = new RelProtoDataType() { + @Override + public RelDataType apply(RelDataTypeFactory a0) { + return a0.builder().add("order_id", SqlTypeName.BIGINT).add("site_id", SqlTypeName.INTEGER) + .add("price", SqlTypeName.DOUBLE).add("order_time", SqlTypeName.TIMESTAMP).build(); + } + }; + + Map consumerPara = new HashMap(); + consumerPara.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + + return new BeamKafkaCSVTable(protoRowType, bootstrapServer, Arrays.asList(topic)) + .updateConsumerProperties(consumerPara); + } +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/package-info.java new file mode 100644 index 000000000000..342a5cd40a12 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ + +/** + * examples on how to use BeamSQL. + * + */ +package org.beam.sdk.java.sql.example; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java index 6d341e9ea06d..b1549b751a14 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java @@ -32,7 +32,7 @@ public interface BeamSQLExpressionExecutor extends Serializable { /** * invoked before data processing. */ - public void prepare(); + void prepare(); /** * apply transformation to input record {@link BeamSQLRow}. @@ -40,7 +40,7 @@ public interface BeamSQLExpressionExecutor extends Serializable { * @param inputRecord * @return */ - public List execute(BeamSQLRow inputRecord); + List execute(BeamSQLRow inputRecord); - public void close(); + void close(); } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java index 9b7c46c9d93f..d759179406d8 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.List; - import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java index 847c1a4920f0..c0d889c9970a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java @@ -20,4 +20,3 @@ * BeamSQL provides a new interface to run a SQL statement with Beam. */ package org.beam.sdk.java.sql; - diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java index d99b7cd0c0ac..73241fd490d6 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java @@ -66,7 +66,8 @@ public class BeamQueryPlanner { protected final Planner planner; private Map sourceTables = new HashMap<>(); - private final JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); + public static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl( + RelDataTypeSystem.DEFAULT); /** * @@ -80,7 +81,7 @@ public BeamQueryPlanner(SchemaPlus schema) { List sqlOperatorTables = new ArrayList<>(); sqlOperatorTables.add(SqlStdOperatorTable.instance()); sqlOperatorTables.add(new CalciteCatalogReader(CalciteSchema.from(schema), false, - Collections.emptyList(), typeFactory)); + Collections.emptyList(), TYPE_FACTORY)); FrameworkConfig config = Frameworks.newConfigBuilder() .parserConfig(SqlParser.configBuilder().setLex(Lex.MYSQL).build()).defaultSchema(schema) @@ -151,7 +152,7 @@ private RelNode convertToBeamRel(RelNode relNode) throws RelConversionException LOG.info("SQLPlan>\n" + RelOptUtil.toString(relNode)); // PlannerImpl.transform() optimizes RelNode with ruleset - return planner.transform(0, traitSet.replace(BeamLogicalConvention.INSTANCE), relNode); + return planner.transform(0, traitSet.plus(BeamLogicalConvention.INSTANCE), relNode); } private RelNode convertToRelNode(SqlNode sqlNode) throws RelConversionException { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java index 814807e37d46..83e03b140a1f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java @@ -32,9 +32,8 @@ import org.beam.sdk.java.sql.rule.BeamProjectRule; /** - * {@link RuleSet} used in {@link BeamQueryPlanner}. - * It translates a standard Calcite {@link RelNode} tree, to - * represent with {@link BeamRelNode} + * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard + * Calcite {@link RelNode} tree, to represent with {@link BeamRelNode} * */ public class BeamRuleSets { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java index 4f4019046b68..1be075992574 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java @@ -63,12 +63,10 @@ public static String explain(final RelNode rel, SqlExplainLevel detailLevel) { try { explain = RelOptUtil.toString(rel); } catch (StackOverflowError e) { - LOG.error( - "StackOverflowError occurred while extracting plan. " + LOG.error("StackOverflowError occurred while extracting plan. " + "Please report it to the dev@ mailing list."); LOG.error("RelNode " + rel + " ExplainLevel " + detailLevel, e); - LOG.error( - "Forcing plan to empty string and continue... " + LOG.error("Forcing plan to empty string and continue... " + "SQL Runner may not working properly after."); } return explain; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java index 0c804d972041..bc3be0a6ccb8 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java @@ -19,10 +19,7 @@ import java.io.Serializable; -import org.apache.calcite.adapter.java.JavaTypeFactory; -import org.apache.calcite.jdbc.JavaTypeFactoryImpl; import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.sql.parser.SqlParseException; @@ -46,7 +43,6 @@ public class BeamSqlRunner implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(BeamSqlRunner.class); - private JavaTypeFactory typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT); private SchemaPlus schema = Frameworks.createRootSchema(true); private BeamQueryPlanner planner = new BeamQueryPlanner(schema); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java index 959996a270ab..6fb2f97722b7 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java @@ -30,7 +30,7 @@ * */ public class BeamFilterRule extends ConverterRule { - public static BeamFilterRule INSTANCE = new BeamFilterRule(); + public static final BeamFilterRule INSTANCE = new BeamFilterRule(); private BeamFilterRule() { super(LogicalFilter.class, Convention.NONE, BeamLogicalConvention.INSTANCE, "BeamFilterRule"); diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java index 73271c393cec..257429b42884 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java @@ -34,7 +34,8 @@ import org.beam.sdk.java.sql.rel.BeamLogicalConvention; /** - * A {@code ConverterRule} to replace {@link TableModify} with {@link BeamIOSinkRel}. + * A {@code ConverterRule} to replace {@link TableModify} with + * {@link BeamIOSinkRel}. * */ public class BeamIOSinkRule extends ConverterRule { @@ -70,7 +71,7 @@ public RelNode convert(RelNode rel) { throw new UnsupportedOperationException( String.format("Streams doesn't support %s modify operation", operation)); } - return new BeamIOSinkRel(cluster, traitSet.replace(BeamLogicalConvention.INSTANCE), + return new BeamIOSinkRel(cluster, traitSet, relOptTable, catalogReader, convertedInput, operation, updateColumnList, sourceExpressionList, flattened); default: diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java index aa364aea256b..a9bb0b5206af 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java @@ -26,7 +26,8 @@ import org.beam.sdk.java.sql.rel.BeamLogicalConvention; /** - * A {@code ConverterRule} to replace {@link TableScan} with {@link BeamIOSourceRel}. + * A {@code ConverterRule} to replace {@link TableScan} with + * {@link BeamIOSourceRel}. * */ public class BeamIOSourceRule extends ConverterRule { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java index 3eee3b29a8e5..d39747602f08 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java @@ -26,7 +26,8 @@ import org.beam.sdk.java.sql.rel.BeamProjectRel; /** - * A {@code ConverterRule} to replace {@link Project} with {@link BeamProjectRel}. + * A {@code ConverterRule} to replace {@link Project} with + * {@link BeamProjectRel}. * */ public class BeamProjectRule extends ConverterRule { diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java index e388cf262ff1..54063b55649a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java @@ -32,6 +32,7 @@ import org.apache.calcite.schema.Schema.TableType; import org.apache.calcite.schema.Statistic; import org.apache.calcite.schema.Statistics; +import org.beam.sdk.java.sql.planner.BeamQueryPlanner; /** * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}. @@ -42,17 +43,13 @@ public abstract class BaseBeamTable implements ScannableTable, Serializable { * */ private static final long serialVersionUID = -1262988061830914193L; - private RelProtoDataType protoRowType; - // // A {@link PTransform} that convert a input record of {@code IO.read()} to - // // {@link BeamSQLRow} - // private PTransform, PCollection> inputTransform; - // // A {@link PTransform} that convert a {@link BeamSQLRow} to the required - // // record of {@code IO.write()} - // private PTransform, PCollection> - // outputTransform; + private RelDataType relDataType; + + protected BeamSQLRecordType beamSqlRecordType; public BaseBeamTable(RelProtoDataType protoRowType) { - this.protoRowType = protoRowType; + this.relDataType = protoRowType.apply(BeamQueryPlanner.TYPE_FACTORY); + this.beamSqlRecordType = BeamSQLRecordType.from(relDataType); } /** @@ -83,7 +80,7 @@ public Enumerable scan(DataContext root) { @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) { - return protoRowType.apply(typeFactory); + return relDataType; } /** diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java index 36734e92498a..03a98d3d17dd 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java @@ -48,6 +48,10 @@ public static BeamSQLRecordType from(RelDataType tableInfo) { return record; } + public int size() { + return fieldsName.size(); + } + public List getFieldsName() { return fieldsName; } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java index 2fb21b1b445d..0ca7b1bd6585 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java @@ -75,6 +75,10 @@ public Object getFieldValue(String fieldName) { dataType.getFieldsType().get(dataType.getFieldsName().indexOf(fieldName))); } + public int size() { + return dataMap.size(); + } + private Object getFieldValue(String fieldName, String fieldType) { if (dataMap.get(fieldName) == null) { return null; @@ -82,6 +86,8 @@ private Object getFieldValue(String fieldName, String fieldType) { switch (SqlTypeName.valueOf(fieldType)) { case INTEGER: return Integer.valueOf(dataMap.get(fieldName)); + case DOUBLE: + return Double.valueOf(dataMap.get(fieldName)); case VARCHAR: return dataMap.get(fieldName); case TIMESTAMP: // TODO diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaCSVTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaCSVTable.java new file mode 100644 index 000000000000..2fda29f3d4e3 --- /dev/null +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaCSVTable.java @@ -0,0 +1,128 @@ +/* + * 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.beam.sdk.java.sql.schema.kafka; + +import java.util.List; + +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.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.beam.sdk.java.sql.schema.BeamSQLRecordType; +import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A Kafka topic that saves records as CSV format. + * + */ +public class BeamKafkaCSVTable extends BeamKafkaTable { + + /** + * + */ + private static final long serialVersionUID = 4754022536543333984L; + + public static final String DELIMITER = ","; + private static final Logger LOG = LoggerFactory.getLogger(BeamKafkaCSVTable.class); + + public BeamKafkaCSVTable(RelProtoDataType protoRowType, String bootstrapServers, + List topics) { + super(protoRowType, bootstrapServers, topics); + } + + @Override + public PTransform>, PCollection> + getPTransformForInput() { + return new CsvRecorderDecoder(beamSqlRecordType); + } + + @Override + public PTransform, PCollection>> + getPTransformForOutput() { + return new CsvRecorderEncoder(beamSqlRecordType); + } + + /** + * A PTransform to convert {@code KV} to {@link BeamSQLRow}. + * + */ + public static class CsvRecorderDecoder + extends PTransform>, PCollection> { + private BeamSQLRecordType recordType; + + public CsvRecorderDecoder(BeamSQLRecordType recordType) { + this.recordType = recordType; + } + + @Override + public PCollection expand(PCollection> input) { + return input.apply("decodeRecord", ParDo.of(new DoFn, BeamSQLRow>() { + @ProcessElement + public void processElement(ProcessContext c) { + String rowInString = new String(c.element().getValue()); + String[] parts = rowInString.split(BeamKafkaCSVTable.DELIMITER); + if (parts.length != recordType.size()) { + LOG.error(String.format("invalid record: ", rowInString)); + } else { + BeamSQLRow sourceRecord = new BeamSQLRow(recordType); + for (int idx = 0; idx < parts.length; ++idx) { + sourceRecord.addField(idx, parts[idx]); + } + c.output(sourceRecord); + } + } + })); + } + } + + /** + * A PTransform to convert {@link BeamSQLRow} to {@code KV}. + * + */ + public static class CsvRecorderEncoder + extends PTransform, PCollection>> { + private BeamSQLRecordType recordType; + + public CsvRecorderEncoder(BeamSQLRecordType recordType) { + this.recordType = recordType; + } + + @Override + public PCollection> expand(PCollection input) { + return input.apply("encodeRecord", ParDo.of(new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) { + BeamSQLRow in = c.element(); + StringBuffer sb = new StringBuffer(); + for (int idx = 0; idx < in.size(); ++idx) { + sb.append(DELIMITER); + sb.append(in.getFieldValue(idx).toString()); + } + c.output(KV.of(new byte[] {}, sb.substring(1).getBytes())); + } + })); + + } + + } + +} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java index ffa8b3aeaaac..878308891633 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java @@ -23,7 +23,10 @@ import java.util.List; import java.util.Map; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -33,11 +36,11 @@ import org.beam.sdk.java.sql.schema.BeamSQLRow; /** - * {@code BeamKafkaTable} represent a Kafka topic, as source or target. - * A user PTransform is required to handle the input/output as KV. + * {@code BeamKafkaTable} represent a Kafka topic, as source or target. Need to + * extend to convert between {@code BeamSQLRow} and {@code KV}. * */ -public class BeamKafkaTable extends BaseBeamTable implements Serializable { +public abstract class BeamKafkaTable extends BaseBeamTable implements Serializable { /** * @@ -69,13 +72,38 @@ public BeamIOType getSourceType() { return BeamIOType.UNBOUNDED; } + /** + * + * @return + */ + public abstract PTransform>, PCollection> + getPTransformForInput(); + + /** + * + * @return + */ + public abstract PTransform, PCollection>> + getPTransformForOutput(); + @Override public PTransform> buildIOReader() { - return null; - // return KafkaIO.read().withBootstrapServers(this.bootstrapServers) - // .withTopics(this.topics).updateConsumerProperties(configUpdates) - // .withKeyCoder(ByteArrayCoder.of()).withValueCoder(ByteArrayCoder.of()).withoutMetadata(); + return new PTransform>() { + /** + * + */ + private static final long serialVersionUID = 9167792271351182771L; + + @Override + public PCollection expand(PBegin input) { + return input.apply("read", + KafkaIO.read().withBootstrapServers(bootstrapServers).withTopics(topics) + .updateConsumerProperties(configUpdates).withKeyCoder(ByteArrayCoder.of()) + .withValueCoder(ByteArrayCoder.of()).withoutMetadata()) + .apply("in_format", getPTransformForInput()); + + } + }; } @Override @@ -83,11 +111,15 @@ public PTransform, PDone> buildIOWriter() { checkArgument(topics != null && topics.size() == 1, "Only one topic can be acceptable as output."); - return null; - // return KafkaIO.write().withBootstrapServers(bootstrapServers) - // .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) - // .withValueCoder(ByteArrayCoder.of()); + return new PTransform, PDone>() { + @Override + public PDone expand(PCollection input) { + return input.apply("out_reformat", getPTransformForOutput()).apply("persistent", + KafkaIO.write().withBootstrapServers(bootstrapServers) + .withTopic(topics.get(0)).withKeyCoder(ByteArrayCoder.of()) + .withValueCoder(ByteArrayCoder.of())); + } + }; } } diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java new file mode 100644 index 000000000000..3cd712159913 --- /dev/null +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java @@ -0,0 +1,41 @@ +package org.beam.sdk.java.sql.planner; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.kafka.BeamKafkaCSVTable; +import org.junit.BeforeClass; + +public class BasePlanner { + public static BeamSqlRunner runner = new BeamSqlRunner(); + + @BeforeClass + public static void prepare(){ + runner.addTable("ORDER_DETAILS", getTable("127.0.0.1:9092", "orders")); + runner.addTable("SUB_ORDER", getTable("127.0.0.1:9092", "sub_orders")); + } + + + public static BaseBeamTable getTable(String bootstrapServer, String topic) { + final RelProtoDataType protoRowType = new RelProtoDataType() { + @Override + public RelDataType apply(RelDataTypeFactory a0) { + return a0.builder().add("order_id", SqlTypeName.BIGINT).add("site_id", SqlTypeName.INTEGER) + .add("price", SqlTypeName.DOUBLE).add("order_time", SqlTypeName.TIMESTAMP).build(); + } + }; + + Map consumerPara = new HashMap(); + consumerPara.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + + return new BeamKafkaCSVTable(protoRowType, bootstrapServer, Arrays.asList(topic)) + .updateConsumerProperties(consumerPara); + } +} diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java new file mode 100644 index 000000000000..df6180ac7e18 --- /dev/null +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java @@ -0,0 +1,16 @@ +package org.beam.sdk.java.sql.planner; + +import org.apache.calcite.sql.parser.SqlParseException; +import org.apache.calcite.tools.RelConversionException; +import org.apache.calcite.tools.ValidationException; +import org.junit.Test; + +public class BeamPlannerExplainTest extends BasePlanner { + + @Test + public void selectAll() throws ValidationException, RelConversionException, SqlParseException{ + String sql = "SELECT * FROM ORDER_DETAILS"; + runner.explainQuery(sql); + } + +} From 3c53db8aa941c3be8ec8171c98cb4405febfac40 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Sun, 12 Mar 2017 19:44:53 -0700 Subject: [PATCH 11/15] add e2e test cases for planner. --- sdks/java/sql/pom.xml | 77 ++++++------ .../sdk/java/sql/planner/BeamSqlRunner.java | 9 +- .../beam/sdk/java/sql/schema/BeamSQLRow.java | 12 ++ .../sdk/java/sql/planner/BasePlanner.java | 37 +++++- .../sql/planner/BeamPlannerExplainTest.java | 62 +++++++++- .../BeamPlannerSubmitTest.java} | 24 +++- .../java/sql/planner/MockedBeamSQLTable.java | 116 ++++++++++++++++++ 7 files changed, 286 insertions(+), 51 deletions(-) rename sdks/java/sql/src/test/java/org/beam/sdk/java/sql/{e2e/package-info.java => planner/BeamPlannerSubmitTest.java} (51%) create mode 100644 sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/MockedBeamSQLTable.java diff --git a/sdks/java/sql/pom.xml b/sdks/java/sql/pom.xml index 1eb3f13da29e..df51da97d2eb 100644 --- a/sdks/java/sql/pom.xml +++ b/sdks/java/sql/pom.xml @@ -40,32 +40,32 @@ - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - - org.apache.maven.plugins - - - maven-checkstyle-plugin - - [2.17,) - - check - - - - - - - - - + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + + org.apache.maven.plugins + + + maven-checkstyle-plugin + + [2.17,) + + check + + + + + + + + + @@ -80,12 +80,7 @@ org.apache.maven.plugins maven-surefire-plugin - - org.apache.beam.sdk.testing.NeedsRunner - - - true - + -da @@ -167,21 +162,21 @@ guava - org.slf4j - slf4j-jdk14 + org.slf4j + slf4j-jdk14 - org.slf4j - slf4j-api + org.slf4j + slf4j-api - org.apache.avro - avro + org.apache.avro + avro - org.apache.calcite - calcite-linq4j - ${calcite-version} + org.apache.calcite + calcite-linq4j + ${calcite-version} diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java index bc3be0a6ccb8..2146082f406b 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java @@ -87,11 +87,18 @@ public void submitQuery(String sqlString) throws Exception { * @throws RelConversionException * @throws SqlParseException */ - public void explainQuery(String sqlString) + public String explainQuery(String sqlString) throws ValidationException, RelConversionException, SqlParseException { BeamRelNode exeTree = planner.convertToBeamRel(sqlString); String beamPlan = RelOptUtil.toString(exeTree); System.out.println(String.format("beamPlan>\n%s", beamPlan)); + planner.planner.close(); + return beamPlan; + } + + protected BeamQueryPlanner getPlanner() { + return planner; } + } diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java index 0ca7b1bd6585..d4b35bbd6a68 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java +++ b/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java @@ -119,6 +119,18 @@ public String toString() { return "RecordRow [dataMap=" + dataMap + ", dataType=" + dataType + "]"; } + /** + * Return data fields as key=value. + * @return + */ + public String valueInString() { + StringBuffer sb = new StringBuffer(); + for (int idx = 0; idx < size(); ++idx) { + sb.append(String.format(",%s=%s", dataType.getFieldsName().get(idx), getFieldValue(idx))); + } + return sb.substring(1); + } + @Override public int hashCode() { final int prime = 31; diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java index 3cd712159913..fcad4cc086c7 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java @@ -1,3 +1,20 @@ +/* + * 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.beam.sdk.java.sql.planner; import java.util.Arrays; @@ -13,15 +30,31 @@ import org.beam.sdk.java.sql.schema.kafka.BeamKafkaCSVTable; import org.junit.BeforeClass; +/** + * prepare {@code BeamSqlRunner} for test. + * + */ public class BasePlanner { public static BeamSqlRunner runner = new BeamSqlRunner(); @BeforeClass - public static void prepare(){ - runner.addTable("ORDER_DETAILS", getTable("127.0.0.1:9092", "orders")); + public static void prepare() { + runner.addTable("ORDER_DETAILS", getTable()); runner.addTable("SUB_ORDER", getTable("127.0.0.1:9092", "sub_orders")); + runner.addTable("SUB_ORDER_RAM", getTable()); } + private static BaseBeamTable getTable() { + final RelProtoDataType protoRowType = new RelProtoDataType() { + @Override + public RelDataType apply(RelDataTypeFactory a0) { + return a0.builder().add("order_id", SqlTypeName.BIGINT).add("site_id", SqlTypeName.INTEGER) + .add("price", SqlTypeName.DOUBLE).add("order_time", SqlTypeName.TIMESTAMP).build(); + } + }; + + return new MockedBeamSQLTable(protoRowType); + } public static BaseBeamTable getTable(String bootstrapServer, String topic) { final RelProtoDataType protoRowType = new RelProtoDataType() { diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java index df6180ac7e18..fe58ef95c4e8 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java @@ -1,16 +1,68 @@ +/* + * 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.beam.sdk.java.sql.planner; -import org.apache.calcite.sql.parser.SqlParseException; -import org.apache.calcite.tools.RelConversionException; -import org.apache.calcite.tools.ValidationException; +import org.junit.Assert; import org.junit.Test; +/** + * Tests to explain queries. + * + */ public class BeamPlannerExplainTest extends BasePlanner { @Test - public void selectAll() throws ValidationException, RelConversionException, SqlParseException{ + public void selectAll() throws Exception { String sql = "SELECT * FROM ORDER_DETAILS"; - runner.explainQuery(sql); + String plan = runner.explainQuery(sql); + + String expectedPlan = + "BeamProjectRel(order_id=[$0], site_id=[$1], price=[$2], order_time=[$3])\n" + + " BeamIOSourceRel(table=[[ORDER_DETAILS]])\n"; + Assert.assertEquals("explain doesn't match", expectedPlan, plan); + } + + @Test + public void selectWithFilter() throws Exception { + String sql = "SELECT " + " order_id, site_id, price " + "FROM ORDER_DETAILS " + + "WHERE SITE_ID = 0 and price > 20"; + String plan = runner.explainQuery(sql); + + String expectedPlan = "BeamProjectRel(order_id=[$0], site_id=[$1], price=[$2])\n" + + " BeamFilterRel(condition=[AND(=($1, 0), >($2, 20))])\n" + + " BeamIOSourceRel(table=[[ORDER_DETAILS]])\n"; + Assert.assertEquals("explain doesn't match", expectedPlan, plan); + } + + @Test + public void insertSelectFilter() throws Exception { + String sql = "INSERT INTO SUB_ORDER(order_id, site_id, price) " + "SELECT " + + " order_id, site_id, price " + "FROM ORDER_DETAILS " + + "WHERE SITE_ID = 0 and price > 20"; + String plan = runner.explainQuery(sql); + + String expectedPlan = + "BeamIOSinkRel(table=[[SUB_ORDER]], operation=[INSERT], flattened=[true])\n" + + " BeamProjectRel(order_id=[$0], site_id=[$1], price=[$2], order_time=[null])\n" + + " BeamProjectRel(order_id=[$0], site_id=[$1], price=[$2])\n" + + " BeamFilterRel(condition=[AND(=($1, 0), >($2, 20))])\n" + + " BeamIOSourceRel(table=[[ORDER_DETAILS]])\n"; + Assert.assertEquals("explain doesn't match", expectedPlan, plan); } } diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerSubmitTest.java similarity index 51% rename from sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java rename to sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerSubmitTest.java index fd380548578a..d3398fb7d73b 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/e2e/package-info.java +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerSubmitTest.java @@ -15,8 +15,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.beam.sdk.java.sql.planner; + +import org.apache.beam.sdk.Pipeline; +import org.junit.Assert; +import org.junit.Test; /** - * contain end-to-end tests. + * Tests to execute a query. + * */ -package org.beam.sdk.java.sql.e2e; +public class BeamPlannerSubmitTest extends BasePlanner { + @Test + public void insertSelectFilter() throws Exception { + String sql = "INSERT INTO SUB_ORDER_RAM(order_id, site_id, price) SELECT " + + " order_id, site_id, price " + "FROM ORDER_DETAILS " + "WHERE SITE_ID = 0 and price > 20"; + Pipeline pipeline = runner.getPlanner().compileBeamPipeline(sql); + runner.getPlanner().planner.close(); + + pipeline.run().waitUntilFinish(); + + Assert.assertTrue(MockedBeamSQLTable.CONTENT.size() == 1); + Assert.assertEquals("order_id=12345,site_id=0,price=20.5", MockedBeamSQLTable.CONTENT.get(0)); + } + +} diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/MockedBeamSQLTable.java b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/MockedBeamSQLTable.java new file mode 100644 index 000000000000..300e459b2948 --- /dev/null +++ b/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/MockedBeamSQLTable.java @@ -0,0 +1,116 @@ +/* + * 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.beam.sdk.java.sql.planner; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.beam.sdk.transforms.Create; +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.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.calcite.rel.type.RelProtoDataType; +import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.sdk.java.sql.schema.BeamIOType; +import org.beam.sdk.java.sql.schema.BeamSQLRow; + +/** + * A mock table use to check input/output. + * + */ +public class MockedBeamSQLTable extends BaseBeamTable { + + /** + * + */ + private static final long serialVersionUID = 1373168368414036932L; + + public static final List CONTENT = new ArrayList<>(); + + public MockedBeamSQLTable(RelProtoDataType protoRowType) { + super(protoRowType); + } + + @Override + public BeamIOType getSourceType() { + return BeamIOType.UNBOUNDED; + } + + @Override + public PTransform> buildIOReader() { + BeamSQLRow row1 = new BeamSQLRow(beamSqlRecordType); + row1.addField(0, 12345L); + row1.addField(1, 0); + row1.addField(2, 10.5); + row1.addField(3, System.currentTimeMillis()); + + BeamSQLRow row2 = new BeamSQLRow(beamSqlRecordType); + row2.addField(0, 12345L); + row2.addField(1, 1); + row2.addField(2, 20.5); + row2.addField(3, System.currentTimeMillis()); + + BeamSQLRow row3 = new BeamSQLRow(beamSqlRecordType); + row3.addField(0, 12345L); + row3.addField(1, 0); + row3.addField(2, 20.5); + row3.addField(3, System.currentTimeMillis()); + + return Create.of(row1, row2, row3); + } + + @Override + public PTransform, PDone> buildIOWriter() { + return new OutputStore(); + } + + /** + * Keep output in {@code CONTENT} for validation. + * + */ + public static class OutputStore extends PTransform, PDone> { + + @Override + public PDone expand(PCollection input) { + input.apply(ParDo.of(new DoFn() { + + @Setup + public void setup() { + CONTENT.clear(); + } + + @ProcessElement + public void processElement(ProcessContext c) { + CONTENT.add(c.element().valueInString()); + } + + @Teardown + public void close() { + + } + + })); + return PDone.in(input.getPipeline()); + } + + } + +} From d5fe32755b626f79abfd950a71e6b46550fa0d2b Mon Sep 17 00:00:00 2001 From: mingmxu Date: Thu, 30 Mar 2017 11:48:50 -0700 Subject: [PATCH 12/15] save work --- "dsls/README.md\n" | 22 +++++++++ dsls/pom.xml | 18 ++++++++ .../README.md\n" => "dsls/sql/README.md\n" | 0 {sdks/java => dsls}/sql/pom.xml | 45 ++++--------------- .../dsls}/sql/example/BeamSqlExample.java | 8 ++-- .../beam/dsls}/sql/example/package-info.java | 2 +- .../BeamSQLExpressionExecutor.java | 5 +-- .../sql/interpreter/BeamSQLSpELExecutor.java | 12 ++--- .../dsls}/sql/interpreter/CalciteToSpEL.java | 4 +- .../dsls}/sql/interpreter/package-info.java | 2 +- .../java/org/beam/dsls}/sql/package-info.java | 2 +- .../sql/planner/BeamPipelineCreator.java | 8 ++-- .../dsls}/sql/planner/BeamQueryPlanner.java | 8 ++-- .../sql/planner/BeamRelDataTypeSystem.java | 2 +- .../beam/dsls}/sql/planner/BeamRuleSets.java | 12 ++--- .../dsls}/sql/planner/BeamSQLRelUtils.java | 4 +- .../beam/dsls}/sql/planner/BeamSqlRunner.java | 6 +-- .../planner/BeamSqlUnsupportedException.java | 2 +- .../planner/UnsupportedOperatorsVisitor.java | 2 +- .../beam/dsls}/sql/planner/package-info.java | 4 +- .../org/beam/dsls}/sql/rel/BeamFilterRel.java | 14 +++--- .../org/beam/dsls}/sql/rel/BeamIOSinkRel.java | 10 ++--- .../beam/dsls}/sql/rel/BeamIOSourceRel.java | 10 ++--- .../dsls}/sql/rel/BeamLogicalConvention.java | 2 +- .../beam/dsls}/sql/rel/BeamProjectRel.java | 16 +++---- .../org/beam/dsls}/sql/rel/BeamRelNode.java | 4 +- .../org/beam/dsls}/sql/rel/package-info.java | 2 +- .../beam/dsls}/sql/rule/BeamFilterRule.java | 6 +-- .../beam/dsls}/sql/rule/BeamIOSinkRule.java | 6 +-- .../beam/dsls}/sql/rule/BeamIOSourceRule.java | 6 +-- .../beam/dsls}/sql/rule/BeamProjectRule.java | 6 +-- .../org/beam/dsls}/sql/rule/package-info.java | 2 +- .../beam/dsls}/sql/schema/BaseBeamTable.java | 4 +- .../org/beam/dsls}/sql/schema/BeamIOType.java | 2 +- .../dsls}/sql/schema/BeamSQLRecordType.java | 2 +- .../org/beam/dsls}/sql/schema/BeamSQLRow.java | 2 +- .../sql/schema/kafka/BeamKafkaCSVTable.java | 6 +-- .../sql/schema/kafka/BeamKafkaTable.java | 8 ++-- .../dsls}/sql/schema/kafka/package-info.java | 2 +- .../beam/dsls}/sql/schema/package-info.java | 2 +- .../dsls}/sql/transform/BeamSQLFilterFn.java | 8 ++-- .../transform/BeamSQLOutputToConsoleFn.java | 4 +- .../dsls}/sql/transform/BeamSQLProjectFn.java | 10 ++--- .../dsls}/sql/transform/package-info.java | 2 +- .../sql/src/main/resources/log4j.properties | 0 .../beam/dsls}/sql/planner/BasePlanner.java | 6 +-- .../sql/planner/BeamPlannerExplainTest.java | 2 +- .../sql/planner/BeamPlannerSubmitTest.java | 2 +- .../dsls}/sql/planner/MockedBeamSQLTable.java | 8 ++-- pom.xml | 14 +++--- sdks/java/pom.xml | 2 +- 51 files changed, 174 insertions(+), 164 deletions(-) create mode 100644 "dsls/README.md\n" create mode 100644 dsls/pom.xml rename "sdks/java/sql/README.md\n" => "dsls/sql/README.md\n" (100%) rename {sdks/java => dsls}/sql/pom.xml (74%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/example/BeamSqlExample.java (94%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/example/package-info.java (95%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/interpreter/BeamSQLExpressionExecutor.java (93%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/interpreter/BeamSQLSpELExecutor.java (93%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/interpreter/CalciteToSpEL.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/interpreter/package-info.java (95%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/package-info.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/BeamPipelineCreator.java (92%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/BeamQueryPlanner.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/BeamRelDataTypeSystem.java (97%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/BeamRuleSets.java (88%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/BeamSQLRelUtils.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/BeamSqlRunner.java (95%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/BeamSqlUnsupportedException.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/UnsupportedOperatorsVisitor.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/planner/package-info.java (88%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rel/BeamFilterRel.java (85%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rel/BeamIOSinkRel.java (91%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rel/BeamIOSourceRel.java (88%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rel/BeamLogicalConvention.java (98%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rel/BeamProjectRel.java (86%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rel/BeamRelNode.java (93%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rel/package-info.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rule/BeamFilterRule.java (92%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rule/BeamIOSinkRule.java (95%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rule/BeamIOSourceRule.java (92%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rule/BeamProjectRule.java (92%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/rule/package-info.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/BaseBeamTable.java (97%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/BeamIOType.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/BeamSQLRecordType.java (98%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/BeamSQLRow.java (99%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/kafka/BeamKafkaCSVTable.java (96%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/kafka/BeamKafkaTable.java (95%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/kafka/package-info.java (95%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/schema/package-info.java (95%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/transform/BeamSQLFilterFn.java (89%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/transform/BeamSQLOutputToConsoleFn.java (93%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/transform/BeamSQLProjectFn.java (88%) rename {sdks/java/sql/src/main/java/org/beam/sdk/java => dsls/sql/src/main/java/org/beam/dsls}/sql/transform/package-info.java (95%) rename {sdks/java => dsls}/sql/src/main/resources/log4j.properties (100%) rename {sdks/java/sql/src/test/java/org/beam/sdk/java => dsls/sql/src/test/java/org/beam/dsls}/sql/planner/BasePlanner.java (94%) rename {sdks/java/sql/src/test/java/org/beam/sdk/java => dsls/sql/src/test/java/org/beam/dsls}/sql/planner/BeamPlannerExplainTest.java (98%) rename {sdks/java/sql/src/test/java/org/beam/sdk/java => dsls/sql/src/test/java/org/beam/dsls}/sql/planner/BeamPlannerSubmitTest.java (97%) rename {sdks/java/sql/src/test/java/org/beam/sdk/java => dsls/sql/src/test/java/org/beam/dsls}/sql/planner/MockedBeamSQLTable.java (94%) diff --git "a/dsls/README.md\n" "b/dsls/README.md\n" new file mode 100644 index 000000000000..24f052f18cf4 --- /dev/null +++ "b/dsls/README.md\n" @@ -0,0 +1,22 @@ + + +# Beam DSLs + +*It's working in progress...* diff --git a/dsls/pom.xml b/dsls/pom.xml new file mode 100644 index 000000000000..bfb2cc103f02 --- /dev/null +++ b/dsls/pom.xml @@ -0,0 +1,18 @@ + + + 4.0.0 + + org.apache.beam + beam-parent + 0.7.0-SNAPSHOT + + beam-dsls-parent + pom + + Apache Beam :: DSLs + + sql + + diff --git "a/sdks/java/sql/README.md\n" "b/dsls/sql/README.md\n" similarity index 100% rename from "sdks/java/sql/README.md\n" rename to "dsls/sql/README.md\n" diff --git a/sdks/java/sql/pom.xml b/dsls/sql/pom.xml similarity index 74% rename from sdks/java/sql/pom.xml rename to dsls/sql/pom.xml index df51da97d2eb..a4ebf942269a 100644 --- a/sdks/java/sql/pom.xml +++ b/dsls/sql/pom.xml @@ -1,15 +1,16 @@ - + 4.0.0 org.apache.beam - beam-sdks-java-parent - 0.6.0-SNAPSHOT + beam-dsls-parent + 0.7.0-SNAPSHOT - - beam-sdk-java-sql - Apache Beam :: SDKs :: Java :: SQL + + beam-dsls-sql + Apache Beam :: DSLs :: SQL Beam SQL provides a new interface to generate a Beam pipeline from SQL statement jar @@ -38,35 +39,6 @@ ${project.basedir}/src/test/ - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - - org.apache.maven.plugins - - - maven-checkstyle-plugin - - [2.17,) - - check - - - - - - - - - - @@ -180,4 +152,3 @@ - diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/BeamSqlExample.java b/dsls/sql/src/main/java/org/beam/dsls/sql/example/BeamSqlExample.java similarity index 94% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/BeamSqlExample.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/example/BeamSqlExample.java index df1032faa168..265c08d01f0a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/BeamSqlExample.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/example/BeamSqlExample.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.example; +package org.beam.dsls.sql.example; import java.io.Serializable; import java.util.Arrays; @@ -27,9 +27,9 @@ import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.beam.sdk.java.sql.planner.BeamSqlRunner; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.kafka.BeamKafkaCSVTable; +import org.beam.dsls.sql.planner.BeamSqlRunner; +import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.kafka.BeamKafkaCSVTable; /** * This is one quick example.
diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/example/package-info.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/example/package-info.java index 342a5cd40a12..ae678e4297fd 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/example/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/example/package-info.java @@ -20,4 +20,4 @@ * examples on how to use BeamSQL. * */ -package org.beam.sdk.java.sql.example; +package org.beam.dsls.sql.example; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLExpressionExecutor.java similarity index 93% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLExpressionExecutor.java index b1549b751a14..56e483ace0e0 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLExpressionExecutor.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLExpressionExecutor.java @@ -15,12 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.interpreter; +package org.beam.dsls.sql.interpreter; import java.io.Serializable; import java.util.List; - -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * {@code BeamSQLExpressionExecutor} fills the gap between relational diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java similarity index 93% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java index c1fbb85eacc1..10efcb4ee7b8 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/BeamSQLSpELExecutor.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.interpreter; +package org.beam.dsls.sql.interpreter; import static com.google.common.base.Preconditions.checkArgument; @@ -26,11 +26,11 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; -import org.beam.sdk.java.sql.planner.BeamSqlUnsupportedException; -import org.beam.sdk.java.sql.rel.BeamFilterRel; -import org.beam.sdk.java.sql.rel.BeamProjectRel; -import org.beam.sdk.java.sql.rel.BeamRelNode; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.planner.BeamSqlUnsupportedException; +import org.beam.dsls.sql.rel.BeamFilterRel; +import org.beam.dsls.sql.rel.BeamProjectRel; +import org.beam.dsls.sql.rel.BeamRelNode; +import org.beam.dsls.sql.schema.BeamSQLRow; import org.springframework.expression.Expression; import org.springframework.expression.ExpressionParser; import org.springframework.expression.spel.SpelParserConfiguration; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java index d759179406d8..721aaae9d5ba 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/CalciteToSpEL.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.interpreter; +package org.beam.dsls.sql.interpreter; import com.google.common.base.Joiner; @@ -25,7 +25,7 @@ import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; -import org.beam.sdk.java.sql.planner.BeamSqlUnsupportedException; +import org.beam.dsls.sql.planner.BeamSqlUnsupportedException; /** * {@code CalciteToSpEL} is used in {@link BeamSQLSpELExecutor}, to convert a diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/package-info.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/package-info.java index 997e16a182b3..85235e2dcb0b 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/interpreter/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/package-info.java @@ -19,4 +19,4 @@ /** * interpreter generate runnable 'code' to execute SQL relational expressions. */ -package org.beam.sdk.java.sql.interpreter; +package org.beam.dsls.sql.interpreter; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/package-info.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/package-info.java index c0d889c9970a..c6f5cf63ba8e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/package-info.java @@ -19,4 +19,4 @@ /** * BeamSQL provides a new interface to run a SQL statement with Beam. */ -package org.beam.sdk.java.sql; +package org.beam.dsls.sql; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java similarity index 92% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java index 71df7b602c5e..59b2806b609a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamPipelineCreator.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import java.util.Map; @@ -23,9 +23,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.values.PCollection; -import org.beam.sdk.java.sql.rel.BeamRelNode; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.rel.BeamRelNode; +import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamQueryPlanner.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamQueryPlanner.java index 73241fd490d6..14656a0e7b30 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamQueryPlanner.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamQueryPlanner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import java.util.ArrayList; import java.util.Collections; @@ -49,9 +49,9 @@ import org.apache.calcite.tools.Planner; import org.apache.calcite.tools.RelConversionException; import org.apache.calcite.tools.ValidationException; -import org.beam.sdk.java.sql.rel.BeamLogicalConvention; -import org.beam.sdk.java.sql.rel.BeamRelNode; -import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.rel.BeamLogicalConvention; +import org.beam.dsls.sql.rel.BeamRelNode; +import org.beam.dsls.sql.schema.BaseBeamTable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRelDataTypeSystem.java similarity index 97% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRelDataTypeSystem.java index 321e2a4c691e..bf35296df648 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRelDataTypeSystem.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRelDataTypeSystem.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.rel.type.RelDataTypeSystemImpl; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRuleSets.java similarity index 88% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRuleSets.java index 83e03b140a1f..39c38bb26268 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamRuleSets.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRuleSets.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -25,11 +25,11 @@ import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.rel.RelNode; import org.apache.calcite.tools.RuleSet; -import org.beam.sdk.java.sql.rel.BeamRelNode; -import org.beam.sdk.java.sql.rule.BeamFilterRule; -import org.beam.sdk.java.sql.rule.BeamIOSinkRule; -import org.beam.sdk.java.sql.rule.BeamIOSourceRule; -import org.beam.sdk.java.sql.rule.BeamProjectRule; +import org.beam.dsls.sql.rel.BeamRelNode; +import org.beam.dsls.sql.rule.BeamFilterRule; +import org.beam.dsls.sql.rule.BeamIOSinkRule; +import org.beam.dsls.sql.rule.BeamIOSourceRule; +import org.beam.dsls.sql.rule.BeamProjectRule; /** * {@link RuleSet} used in {@link BeamQueryPlanner}. It translates a standard diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java index 1be075992574..f5157acc11dd 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSQLRelUtils.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import java.util.concurrent.atomic.AtomicInteger; @@ -23,7 +23,7 @@ import org.apache.calcite.plan.volcano.RelSubset; import org.apache.calcite.rel.RelNode; import org.apache.calcite.sql.SqlExplainLevel; -import org.beam.sdk.java.sql.rel.BeamRelNode; +import org.beam.dsls.sql.rel.BeamRelNode; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java index 2146082f406b..3cc5a0f8b659 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlRunner.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import java.io.Serializable; @@ -26,8 +26,8 @@ import org.apache.calcite.tools.Frameworks; import org.apache.calcite.tools.RelConversionException; import org.apache.calcite.tools.ValidationException; -import org.beam.sdk.java.sql.rel.BeamRelNode; -import org.beam.sdk.java.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.rel.BeamRelNode; +import org.beam.dsls.sql.schema.BaseBeamTable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlUnsupportedException.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlUnsupportedException.java index 1aa2c7ceefd2..a3475bb1c261 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/BeamSqlUnsupportedException.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlUnsupportedException.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; /** * Generic exception for un-supported operations. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/UnsupportedOperatorsVisitor.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/UnsupportedOperatorsVisitor.java index c95804395d29..702381df60be 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/UnsupportedOperatorsVisitor.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/UnsupportedOperatorsVisitor.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import org.apache.calcite.sql.util.SqlShuttle; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/package-info.java similarity index 88% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/planner/package-info.java index b62bf67aed51..d98c58424121 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/planner/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/package-info.java @@ -17,8 +17,8 @@ */ /** - * {@link org.beam.sdk.java.sql.planner.BeamQueryPlanner} is the main interface. + * {@link org.beam.dsls.sql.planner.BeamQueryPlanner} is the main interface. * It defines data sources, validate a SQL statement, and convert it as a Beam * pipeline. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamFilterRel.java similarity index 85% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamFilterRel.java index c5cac6d05f87..64f2d1fa6700 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamFilterRel.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamFilterRel.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rel; +package org.beam.dsls.sql.rel; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.ParDo; @@ -25,12 +25,12 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Filter; import org.apache.calcite.rex.RexNode; -import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; -import org.beam.sdk.java.sql.interpreter.BeamSQLSpELExecutor; -import org.beam.sdk.java.sql.planner.BeamPipelineCreator; -import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; -import org.beam.sdk.java.sql.schema.BeamSQLRow; -import org.beam.sdk.java.sql.transform.BeamSQLFilterFn; +import org.beam.dsls.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.dsls.sql.interpreter.BeamSQLSpELExecutor; +import org.beam.dsls.sql.planner.BeamPipelineCreator; +import org.beam.dsls.sql.planner.BeamSQLRelUtils; +import org.beam.dsls.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.transform.BeamSQLFilterFn; /** * BeamRelNode to replace a {@code Filter} node. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java similarity index 91% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java index 452cf7c5f8dc..9e8bb0a4fc10 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSinkRel.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rel; +package org.beam.dsls.sql.rel; import com.google.common.base.Joiner; @@ -30,10 +30,10 @@ import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.TableModify; import org.apache.calcite.rex.RexNode; -import org.beam.sdk.java.sql.planner.BeamPipelineCreator; -import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.planner.BeamPipelineCreator; +import org.beam.dsls.sql.planner.BeamSQLRelUtils; +import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * BeamRelNode to replace a {@code TableModify} node. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java similarity index 88% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java index f9ac35cdda7c..a7713459cc9e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamIOSourceRel.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rel; +package org.beam.dsls.sql.rel; import com.google.common.base.Joiner; @@ -25,10 +25,10 @@ import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.RelTraitSet; import org.apache.calcite.rel.core.TableScan; -import org.beam.sdk.java.sql.planner.BeamPipelineCreator; -import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.planner.BeamPipelineCreator; +import org.beam.dsls.sql.planner.BeamSQLRelUtils; +import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * BeamRelNode to replace a {@code TableScan} node. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamLogicalConvention.java similarity index 98% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamLogicalConvention.java index 739bc012189b..50fe8e013af1 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamLogicalConvention.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamLogicalConvention.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rel; +package org.beam.dsls.sql.rel; import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.ConventionTraitDef; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java similarity index 86% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java index 9086c6c0f593..8519294e1e70 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamProjectRel.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rel; +package org.beam.dsls.sql.rel; import java.util.List; @@ -31,13 +31,13 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; -import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; -import org.beam.sdk.java.sql.interpreter.BeamSQLSpELExecutor; -import org.beam.sdk.java.sql.planner.BeamPipelineCreator; -import org.beam.sdk.java.sql.planner.BeamSQLRelUtils; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import org.beam.sdk.java.sql.schema.BeamSQLRow; -import org.beam.sdk.java.sql.transform.BeamSQLProjectFn; +import org.beam.dsls.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.dsls.sql.interpreter.BeamSQLSpELExecutor; +import org.beam.dsls.sql.planner.BeamPipelineCreator; +import org.beam.dsls.sql.planner.BeamSQLRelUtils; +import org.beam.dsls.sql.schema.BeamSQLRecordType; +import org.beam.dsls.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.transform.BeamSQLProjectFn; /** * BeamRelNode to replace a {@code Project} node. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamRelNode.java similarity index 93% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamRelNode.java index 65bab08c7855..4b53943d891f 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/BeamRelNode.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamRelNode.java @@ -15,11 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rel; +package org.beam.dsls.sql.rel; import org.apache.beam.sdk.Pipeline; import org.apache.calcite.rel.RelNode; -import org.beam.sdk.java.sql.planner.BeamPipelineCreator; +import org.beam.dsls.sql.planner.BeamPipelineCreator; /** * A new method {@link #buildBeamPipeline(BeamPipelineCreator)} is added, it's diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/package-info.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rel/package-info.java index bd3cf1d96151..13dc96285942 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rel/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/package-info.java @@ -20,4 +20,4 @@ * BeamSQL specified nodes, to replace {@link org.apache.calcite.rel.RelNode}. * */ -package org.beam.sdk.java.sql.rel; +package org.beam.dsls.sql.rel; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamFilterRule.java similarity index 92% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamFilterRule.java index 6fb2f97722b7..2ad7c074dbdc 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamFilterRule.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamFilterRule.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rule; +package org.beam.dsls.sql.rule; import org.apache.calcite.plan.Convention; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; import org.apache.calcite.rel.core.Filter; import org.apache.calcite.rel.logical.LogicalFilter; -import org.beam.sdk.java.sql.rel.BeamFilterRel; -import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +import org.beam.dsls.sql.rel.BeamFilterRel; +import org.beam.dsls.sql.rel.BeamLogicalConvention; /** * A {@code ConverterRule} to replace {@link Filter} with {@link BeamFilterRel}. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java index 257429b42884..232269c53ef7 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSinkRule.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rule; +package org.beam.dsls.sql.rule; import java.util.List; @@ -30,8 +30,8 @@ import org.apache.calcite.rel.logical.LogicalTableModify; import org.apache.calcite.rex.RexNode; import org.apache.calcite.schema.Table; -import org.beam.sdk.java.sql.rel.BeamIOSinkRel; -import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +import org.beam.dsls.sql.rel.BeamIOSinkRel; +import org.beam.dsls.sql.rel.BeamLogicalConvention; /** * A {@code ConverterRule} to replace {@link TableModify} with diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSourceRule.java similarity index 92% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSourceRule.java index a9bb0b5206af..9e4778b5ff04 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamIOSourceRule.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSourceRule.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rule; +package org.beam.dsls.sql.rule; import org.apache.calcite.plan.Convention; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; import org.apache.calcite.rel.core.TableScan; import org.apache.calcite.rel.logical.LogicalTableScan; -import org.beam.sdk.java.sql.rel.BeamIOSourceRel; -import org.beam.sdk.java.sql.rel.BeamLogicalConvention; +import org.beam.dsls.sql.rel.BeamIOSourceRel; +import org.beam.dsls.sql.rel.BeamLogicalConvention; /** * A {@code ConverterRule} to replace {@link TableScan} with diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamProjectRule.java similarity index 92% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamProjectRule.java index d39747602f08..117a056fa0ff 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/BeamProjectRule.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamProjectRule.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.rule; +package org.beam.dsls.sql.rule; import org.apache.calcite.plan.Convention; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.convert.ConverterRule; import org.apache.calcite.rel.core.Project; import org.apache.calcite.rel.logical.LogicalProject; -import org.beam.sdk.java.sql.rel.BeamLogicalConvention; -import org.beam.sdk.java.sql.rel.BeamProjectRel; +import org.beam.dsls.sql.rel.BeamLogicalConvention; +import org.beam.dsls.sql.rel.BeamProjectRel; /** * A {@code ConverterRule} to replace {@link Project} with diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/package-info.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/rule/package-info.java index d363949c8933..634f6a8b37ec 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/rule/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/package-info.java @@ -19,4 +19,4 @@ /** * {@link RelOptRule} to generate {@link BeamRelNode}. */ -package org.beam.sdk.java.sql.rule; +package org.beam.dsls.sql.rule; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BaseBeamTable.java similarity index 97% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/BaseBeamTable.java index 54063b55649a..5a815c81ab21 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BaseBeamTable.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BaseBeamTable.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.schema; +package org.beam.dsls.sql.schema; import java.io.Serializable; @@ -30,9 +30,9 @@ import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.schema.ScannableTable; import org.apache.calcite.schema.Schema.TableType; +import org.beam.dsls.sql.planner.BeamQueryPlanner; import org.apache.calcite.schema.Statistic; import org.apache.calcite.schema.Statistics; -import org.beam.sdk.java.sql.planner.BeamQueryPlanner; /** * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamIOType.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamIOType.java index 9079cb92ada6..5e55b0fc6e48 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamIOType.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamIOType.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.schema; +package org.beam.dsls.sql.schema; import java.io.Serializable; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java similarity index 98% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java index 03a98d3d17dd..250cb0ccf41a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRecordType.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.schema; +package org.beam.dsls.sql.schema; import java.io.Serializable; import java.util.ArrayList; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java similarity index 99% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java index d4b35bbd6a68..740ec487ea8d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/BeamSQLRow.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.schema; +package org.beam.dsls.sql.schema; import java.io.Serializable; import java.util.HashMap; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaCSVTable.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java similarity index 96% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaCSVTable.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java index 2fda29f3d4e3..2c5df88bf01e 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaCSVTable.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.schema.kafka; +package org.beam.dsls.sql.schema.kafka; import java.util.List; @@ -25,8 +25,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.rel.type.RelProtoDataType; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.schema.BeamSQLRecordType; +import org.beam.dsls.sql.schema.BeamSQLRow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java index 878308891633..05178ce02e8a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/BeamKafkaTable.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.schema.kafka; +package org.beam.dsls.sql.schema.kafka; import static com.google.common.base.Preconditions.checkArgument; @@ -31,9 +31,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.calcite.rel.type.RelProtoDataType; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.BeamIOType; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.BeamIOType; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * {@code BeamKafkaTable} represent a Kafka topic, as source or target. Need to diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/package-info.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/package-info.java index 737fd20fe258..822fce703da1 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/kafka/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/package-info.java @@ -19,4 +19,4 @@ /** * table schema for KafkaIO. */ -package org.beam.sdk.java.sql.schema.kafka; +package org.beam.dsls.sql.schema.kafka; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/package-info.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/schema/package-info.java index 046a51269e2a..ef9cc7d38b6a 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/schema/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/package-info.java @@ -20,4 +20,4 @@ * define table schema, to map with Beam IO components. * */ -package org.beam.sdk.java.sql.schema; +package org.beam.dsls.sql.schema; diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java similarity index 89% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java index e46c0515f916..6a7fb366b68d 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLFilterFn.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.transform; +package org.beam.dsls.sql.transform; import java.util.List; import org.apache.beam.sdk.transforms.DoFn; -import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; -import org.beam.sdk.java.sql.rel.BeamFilterRel; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.dsls.sql.rel.BeamFilterRel; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * {@code BeamSQLFilterFn} is the executor for a {@link BeamFilterRel} step. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLOutputToConsoleFn.java similarity index 93% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLOutputToConsoleFn.java index 2748424fd1ea..06150fc1fec7 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLOutputToConsoleFn.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLOutputToConsoleFn.java @@ -15,10 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.transform; +package org.beam.dsls.sql.transform; import org.apache.beam.sdk.transforms.DoFn; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * A test PTransform to display output in console. diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java similarity index 88% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java index 92fc66f8fab2..2ad79ccca7c2 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/BeamSQLProjectFn.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.transform; +package org.beam.dsls.sql.transform; import java.util.List; import org.apache.beam.sdk.transforms.DoFn; -import org.beam.sdk.java.sql.interpreter.BeamSQLExpressionExecutor; -import org.beam.sdk.java.sql.rel.BeamProjectRel; -import org.beam.sdk.java.sql.schema.BeamSQLRecordType; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.interpreter.BeamSQLExpressionExecutor; +import org.beam.dsls.sql.rel.BeamProjectRel; +import org.beam.dsls.sql.schema.BeamSQLRecordType; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * diff --git a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/package-info.java similarity index 95% rename from sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java rename to dsls/sql/src/main/java/org/beam/dsls/sql/transform/package-info.java index ddd4d1667f29..91b5639c6225 100644 --- a/sdks/java/sql/src/main/java/org/beam/sdk/java/sql/transform/package-info.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/package-info.java @@ -19,4 +19,4 @@ /** * {@link PTransform} used in a BeamSQL pipeline. */ -package org.beam.sdk.java.sql.transform; +package org.beam.dsls.sql.transform; diff --git a/sdks/java/sql/src/main/resources/log4j.properties b/dsls/sql/src/main/resources/log4j.properties similarity index 100% rename from sdks/java/sql/src/main/resources/log4j.properties rename to dsls/sql/src/main/resources/log4j.properties diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BasePlanner.java similarity index 94% rename from sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java rename to dsls/sql/src/test/java/org/beam/dsls/sql/planner/BasePlanner.java index fcad4cc086c7..56e45c4732d2 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BasePlanner.java +++ b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BasePlanner.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import java.util.Arrays; import java.util.HashMap; @@ -26,8 +26,8 @@ import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.kafka.BeamKafkaCSVTable; +import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.kafka.BeamKafkaCSVTable; import org.junit.BeforeClass; /** diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerExplainTest.java similarity index 98% rename from sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java rename to dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerExplainTest.java index fe58ef95c4e8..a77878fc150e 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerExplainTest.java +++ b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerExplainTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import org.junit.Assert; import org.junit.Test; diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerSubmitTest.java b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerSubmitTest.java similarity index 97% rename from sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerSubmitTest.java rename to dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerSubmitTest.java index d3398fb7d73b..c49bcafe90cd 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/BeamPlannerSubmitTest.java +++ b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerSubmitTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import org.apache.beam.sdk.Pipeline; import org.junit.Assert; diff --git a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/MockedBeamSQLTable.java b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/MockedBeamSQLTable.java similarity index 94% rename from sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/MockedBeamSQLTable.java rename to dsls/sql/src/test/java/org/beam/dsls/sql/planner/MockedBeamSQLTable.java index 300e459b2948..5924493f51f7 100644 --- a/sdks/java/sql/src/test/java/org/beam/sdk/java/sql/planner/MockedBeamSQLTable.java +++ b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/MockedBeamSQLTable.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.beam.sdk.java.sql.planner; +package org.beam.dsls.sql.planner; import java.util.ArrayList; import java.util.List; @@ -28,9 +28,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.calcite.rel.type.RelProtoDataType; -import org.beam.sdk.java.sql.schema.BaseBeamTable; -import org.beam.sdk.java.sql.schema.BeamIOType; -import org.beam.sdk.java.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.BeamIOType; +import org.beam.dsls.sql.schema.BeamSQLRow; /** * A mock table use to check input/output. diff --git a/pom.xml b/pom.xml index c3b847648f76..da9ac09d9a71 100644 --- a/pom.xml +++ b/pom.xml @@ -14,8 +14,7 @@ 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. ---> - +--> 4.0.0 @@ -92,13 +91,13 @@ UTF-8 - + true - + 3.5 @@ -153,6 +152,7 @@ examples sdks/java/javadoc + dsls @@ -1285,7 +1285,7 @@ - + @@ -1299,7 +1299,7 @@ - + @@ -1487,4 +1487,4 @@ 3.2 - + \ No newline at end of file diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index eb07bfe0c362..5180352cfc31 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -42,7 +42,7 @@ - sql + From c96fa7bd1da706031cb99f81911c3f01eee86028 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Sat, 8 Apr 2017 21:10:31 -0700 Subject: [PATCH 13/15] refine BeamSQLRow; --- .../dsls/sql/planner/BeamPipelineCreator.java | 7 + .../dsls/sql/schema/BeamSQLRecordType.java | 11 +- .../sql/schema/BeamSQLRecordTypeCoder.java | 74 +++++++ .../org/beam/dsls/sql/schema/BeamSQLRow.java | 192 +++++++++++++----- .../beam/dsls/sql/schema/BeamSqlRowCoder.java | 138 +++++++++++++ .../sql/schema/InvalidFieldException.java | 13 ++ .../schema/UnsupportedDataTypeException.java | 11 + .../transform/BeamSQLOutputToConsoleFn.java | 2 +- .../sql/planner/BeamPlannerSubmitTest.java | 2 +- .../dsls/sql/planner/MockedBeamSQLTable.java | 13 +- 10 files changed, 402 insertions(+), 61 deletions(-) create mode 100644 dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java create mode 100644 dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java create mode 100644 dsls/sql/src/main/java/org/beam/dsls/sql/schema/InvalidFieldException.java create mode 100644 dsls/sql/src/main/java/org/beam/dsls/sql/schema/UnsupportedDataTypeException.java diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java index 59b2806b609a..826c6a568b1c 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java @@ -20,12 +20,16 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.values.PCollection; import org.beam.dsls.sql.rel.BeamRelNode; import org.beam.dsls.sql.schema.BaseBeamTable; +import org.beam.dsls.sql.schema.BeamSQLRecordType; +import org.beam.dsls.sql.schema.BeamSQLRecordTypeCoder; import org.beam.dsls.sql.schema.BeamSQLRow; +import org.beam.dsls.sql.schema.BeamSqlRowCoder; /** * {@link BeamPipelineCreator} converts a {@link BeamRelNode} tree, into a Beam @@ -50,6 +54,9 @@ public BeamPipelineCreator(Map sourceTables) { options.setJobName("BeamPlanCreator"); pipeline = Pipeline.create(options); + CoderRegistry cr = pipeline.getCoderRegistry(); + cr.registerCoder(BeamSQLRow.class, BeamSqlRowCoder.of()); + cr.registerCoder(BeamSQLRecordType.class, BeamSQLRecordTypeCoder.of()); } public PCollection getLatestStream() { diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java index 250cb0ccf41a..d2dd9854285c 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java @@ -25,25 +25,26 @@ import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.sql.type.SqlTypeName; /** * Field type information in {@link BeamSQLRow}. * */ -@DefaultCoder(AvroCoder.class) +//@DefaultCoder(BeamSQLRecordTypeCoder.class) public class BeamSQLRecordType implements Serializable { /** * */ private static final long serialVersionUID = -5318734648766104712L; private List fieldsName = new ArrayList<>(); - private List fieldsType = new ArrayList<>(); + private List fieldsType = new ArrayList<>(); public static BeamSQLRecordType from(RelDataType tableInfo) { BeamSQLRecordType record = new BeamSQLRecordType(); for (RelDataTypeField f : tableInfo.getFieldList()) { record.fieldsName.add(f.getName()); - record.fieldsType.add(f.getType().getSqlTypeName().getName()); + record.fieldsType.add(f.getType().getSqlTypeName()); } return record; } @@ -60,11 +61,11 @@ public void setFieldsName(List fieldsName) { this.fieldsName = fieldsName; } - public List getFieldsType() { + public List getFieldsType() { return fieldsType; } - public void setFieldsType(List fieldsType) { + public void setFieldsType(List fieldsType) { this.fieldsType = fieldsType; } diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java new file mode 100644 index 000000000000..c30f39e5d8e6 --- /dev/null +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java @@ -0,0 +1,74 @@ +package org.beam.dsls.sql.schema; + +import com.fasterxml.jackson.annotation.JsonCreator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.sdk.coders.Coder.Context; + +/** + * A {@link Coder} for {@link BeamSQLRecordType}. + * + */ +public class BeamSQLRecordTypeCoder extends StandardCoder { + private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); + private static final VarIntCoder intCoder = VarIntCoder.of(); + + private static final BeamSQLRecordTypeCoder INSTANCE = new BeamSQLRecordTypeCoder(); + private BeamSQLRecordTypeCoder(){} + + @JsonCreator + public static BeamSQLRecordTypeCoder of() { + return INSTANCE; + } + + @Override + public void encode(BeamSQLRecordType value, OutputStream outStream, + org.apache.beam.sdk.coders.Coder.Context context) throws CoderException, IOException { + Context nested = context.nested(); + intCoder.encode(value.size(), outStream, nested); + for(String fieldName : value.getFieldsName()){ + stringCoder.encode(fieldName, outStream, nested); + } + for(SqlTypeName fieldType : value.getFieldsType()){ + stringCoder.encode(fieldType.name(), outStream, nested); + } + outStream.flush(); + } + + @Override + public BeamSQLRecordType decode(InputStream inStream, + org.apache.beam.sdk.coders.Coder.Context context) throws CoderException, IOException { + BeamSQLRecordType typeRecord = new BeamSQLRecordType(); + Context nested = context.nested(); + int size = intCoder.decode(inStream, nested); + for(int idx=0; idx> getCoderArguments() { + // TODO Auto-generated method stub + return null; + } + + @Override + public void verifyDeterministic() + throws org.apache.beam.sdk.coders.Coder.NonDeterministicException { + // TODO Auto-generated method stub + + } + +} diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java index 740ec487ea8d..d74d84cbeb2a 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java @@ -18,92 +18,183 @@ package org.beam.dsls.sql.schema; import java.io.Serializable; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Date; import java.util.HashMap; +import java.util.List; import java.util.Map; -import org.apache.avro.reflect.Nullable; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.calcite.sql.type.SqlTypeName; /** * Repersent a generic ROW record in Beam SQL. * */ -@DefaultCoder(AvroCoder.class) +//@DefaultCoder(BeamSqlRowCoder.class) public class BeamSQLRow implements Serializable { /** * */ private static final long serialVersionUID = 4569220242480160895L; - private Map dataMap = new HashMap<>(); - @Nullable + private List nullFields = new ArrayList<>(); + private List dataValues; private BeamSQLRecordType dataType; - @Deprecated - public BeamSQLRow() { + public BeamSQLRow(BeamSQLRecordType dataType) { + this.dataType = dataType; + this.dataValues = new ArrayList<>(); + for(int idx=0; idx dataValues) { + this.dataValues = dataValues; this.dataType = dataType; } public void addField(String fieldName, Object fieldValue) { - if (fieldValue != null) { - dataMap.put(fieldName, fieldValue.toString()); - } else { - // dataMap.put(fieldName, null); - } + addField(dataType.getFieldsName().indexOf(fieldName), fieldValue); } public void addField(int index, Object fieldValue) { - addField(dataType.getFieldsName().get(index), fieldValue); + if(fieldValue == null){ + dataValues.set(index, fieldValue); + if(!nullFields.contains(index)){nullFields.add(index);} + return; + } + + SqlTypeName fieldType = dataType.getFieldsType().get(index); + switch (fieldType) { + case INTEGER: + case SMALLINT: + case TINYINT: + if(!(fieldValue instanceof Integer)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + } + break; + case DOUBLE: + if(!(fieldValue instanceof Double)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + } + break; + case BIGINT: + if(!(fieldValue instanceof Long)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + } + break; + case FLOAT: + if(!(fieldValue instanceof Float)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + } + break; + case VARCHAR: + if(!(fieldValue instanceof String)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + } + break; + case TIME: + case TIMESTAMP: + if(!(fieldValue instanceof Date)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + } + break; + default: + throw new UnsupportedDataTypeException(fieldType); + } + dataValues.set(index, fieldValue); } - public Object getFieldValue(int fieldIdx) { - return getFieldValue(dataType.getFieldsName().get(fieldIdx), - dataType.getFieldsType().get(fieldIdx)); + + public int getInteger(int idx) { + return (Integer) getFieldValue(idx); } - public Object getFieldValue(String fieldName) { - if (dataType.getFieldsName().indexOf(fieldName) == -1) { - return null; - } - return getFieldValue(fieldName, - dataType.getFieldsType().get(dataType.getFieldsName().indexOf(fieldName))); + public double getDouble(int idx) { + return (Double) getFieldValue(idx); } - public int size() { - return dataMap.size(); + public long getLong(int idx) { + return (Long) getFieldValue(idx); + } + + public String getString(int idx) { + return (String) getFieldValue(idx); } - private Object getFieldValue(String fieldName, String fieldType) { - if (dataMap.get(fieldName) == null) { + public Date getDate(int idx) { + return (Date) getFieldValue(idx); + } + + public Object getFieldValue(String fieldName) { + return getFieldValue(dataType.getFieldsName().indexOf(fieldName)); + } + + public Object getFieldValue(int fieldIdx) { + if(nullFields.contains(fieldIdx)){ return null; } - switch (SqlTypeName.valueOf(fieldType)) { + + Object fieldValue = dataValues.get(fieldIdx); + SqlTypeName fieldType = dataType.getFieldsType().get(fieldIdx); + + switch (fieldType) { case INTEGER: - return Integer.valueOf(dataMap.get(fieldName)); + case SMALLINT: + case TINYINT: + if(!(fieldValue instanceof Integer)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + }else{ + return Integer.valueOf(fieldValue.toString()); + } case DOUBLE: - return Double.valueOf(dataMap.get(fieldName)); - case VARCHAR: - return dataMap.get(fieldName); - case TIMESTAMP: // TODO + if(!(fieldValue instanceof Double)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + }else{ + return Double.valueOf(fieldValue.toString()); + } case BIGINT: - return Long.valueOf(dataMap.get(fieldName)); + if(!(fieldValue instanceof Long)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + }else{ + return Long.valueOf(fieldValue.toString()); + } + case FLOAT: + if(!(fieldValue instanceof Float)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + }else{ + return Float.valueOf(fieldValue.toString()); + } + case VARCHAR: + if(!(fieldValue instanceof String)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + }else{ + return fieldValue.toString(); + } + case TIME: + case TIMESTAMP: + if(!(fieldValue instanceof Date)){ + throw new InvalidFieldException(String.format("[%s] doesn't match type [%s]", fieldValue, fieldType)); + }else{ + return fieldValue; + } default: - return dataMap.get(fieldName); + throw new UnsupportedDataTypeException(fieldType); } } - public Map getDataMap() { - return dataMap; + public int size() { + return dataValues.size(); + } + + public List getDataValues() { + return dataValues; } - public void setDataMap(HashMap dataMap) { - this.dataMap = dataMap; + public void setDataValues(List dataValues) { + this.dataValues = dataValues; } public BeamSQLRecordType getDataType() { @@ -114,9 +205,17 @@ public void setDataType(BeamSQLRecordType dataType) { this.dataType = dataType; } + public void setNullFields(List nullFields) { + this.nullFields = nullFields; + } + + public List getNullFields() { + return nullFields; + } + @Override public String toString() { - return "RecordRow [dataMap=" + dataMap + ", dataType=" + dataType + "]"; + return "BeamSQLRow [dataValues=" + dataValues + ", dataType=" + dataType + "]"; } /** @@ -131,15 +230,6 @@ public String valueInString() { return sb.substring(1); } - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((dataMap == null) ? 0 : dataMap.hashCode()); - result = prime * result + ((dataType == null) ? 0 : dataType.hashCode()); - return result; - } - @Override public boolean equals(Object obj) { if (this == obj) { diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java new file mode 100644 index 000000000000..a93ca25de372 --- /dev/null +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java @@ -0,0 +1,138 @@ +package org.beam.dsls.sql.schema; + +import com.fasterxml.jackson.annotation.JsonCreator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Date; +import java.util.List; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.BigIntegerCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.DoubleCoder; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.MapCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.Coder.Context; + +/** + * A {@link Coder} encodes {@link BeamSQLRow}. + * + */ +public class BeamSqlRowCoder extends StandardCoder{ + private static final BeamSQLRecordTypeCoder recordTypeCoder = BeamSQLRecordTypeCoder.of(); + + private static final ListCoder listCoder = ListCoder.of(BigEndianIntegerCoder.of()); + + private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); + private static final BigEndianIntegerCoder intCoder = BigEndianIntegerCoder.of(); + private static final BigEndianLongCoder longCoder = BigEndianLongCoder.of(); + private static final DoubleCoder doubleCoder = DoubleCoder.of(); + + private static final BeamSqlRowCoder INSTANCE = new BeamSqlRowCoder(); + private BeamSqlRowCoder(){} + + @JsonCreator + public static BeamSqlRowCoder of() { + return INSTANCE; + } + + @Override + public void encode(BeamSQLRow value, OutputStream outStream, + org.apache.beam.sdk.coders.Coder.Context context) throws CoderException, IOException { + recordTypeCoder.encode(value.getDataType(), outStream, context); + listCoder.encode(value.getNullFields(), outStream, context); + + Context nested = context.nested(); + + for (int idx = 0; idx < value.size(); ++idx) { + if(value.getNullFields().contains(idx)){ + continue; + } + + switch (value.getDataType().getFieldsType().get(idx)) { + case INTEGER: + case SMALLINT: + case TINYINT: + intCoder.encode(value.getInteger(idx), outStream, nested); + break; + case DOUBLE: + case FLOAT: + doubleCoder.encode(value.getDouble(idx), outStream, nested); + break; + case BIGINT: + longCoder.encode(value.getLong(idx), outStream, nested); + break; + case VARCHAR: + stringCoder.encode(value.getString(idx), outStream, nested); + break; + case TIME: + case TIMESTAMP: + longCoder.encode(value.getDate(idx).getTime(), outStream, nested); + break; + + default: + throw new UnsupportedDataTypeException(value.getDataType().getFieldsType().get(idx)); + } + } + } + + @Override + public BeamSQLRow decode(InputStream inStream, org.apache.beam.sdk.coders.Coder.Context context) + throws CoderException, IOException { + BeamSQLRecordType type = recordTypeCoder.decode(inStream, context); + List nullFields = listCoder.decode(inStream, context); + + BeamSQLRow record = new BeamSQLRow(type); + record.setNullFields(nullFields); + + for (int idx = 0; idx < type.size(); ++idx) { + if(nullFields.contains(idx)){ + continue; + } + + switch (type.getFieldsType().get(idx)) { + case INTEGER: + case SMALLINT: + case TINYINT: + record.addField(idx, intCoder.decode(inStream, context)); + break; + case DOUBLE: + case FLOAT: + record.addField(idx, doubleCoder.decode(inStream, context)); + break; + case BIGINT: + record.addField(idx, longCoder.decode(inStream, context)); + break; + case VARCHAR: + record.addField(idx, stringCoder.decode(inStream, context)); + break; + case TIME: + case TIMESTAMP: + record.addField(idx, new Date(longCoder.decode(inStream, context))); + break; + + default: + throw new UnsupportedDataTypeException(type.getFieldsType().get(idx)); + } + } + + return record; + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public void verifyDeterministic() + throws org.apache.beam.sdk.coders.Coder.NonDeterministicException { + + } + +} diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/InvalidFieldException.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/InvalidFieldException.java new file mode 100644 index 000000000000..3ab86c52e1e1 --- /dev/null +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/InvalidFieldException.java @@ -0,0 +1,13 @@ +package org.beam.dsls.sql.schema; + +public class InvalidFieldException extends RuntimeException { + + public InvalidFieldException() { + super(); + } + + public InvalidFieldException(String message) { + super(message); + } + +} diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/UnsupportedDataTypeException.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/UnsupportedDataTypeException.java new file mode 100644 index 000000000000..7f7afb23991e --- /dev/null +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/UnsupportedDataTypeException.java @@ -0,0 +1,11 @@ +package org.beam.dsls.sql.schema; + +import org.apache.calcite.sql.type.SqlTypeName; + +public class UnsupportedDataTypeException extends RuntimeException { + + public UnsupportedDataTypeException(SqlTypeName unsupportedType){ + super(String.format("Not support data type [%s]", unsupportedType)); + } + +} diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLOutputToConsoleFn.java b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLOutputToConsoleFn.java index 06150fc1fec7..1014c0d3f7b1 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLOutputToConsoleFn.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLOutputToConsoleFn.java @@ -39,7 +39,7 @@ public BeamSQLOutputToConsoleFn(String stepName) { @ProcessElement public void processElement(ProcessContext c) { - System.out.println("Output: " + c.element().getDataMap()); + System.out.println("Output: " + c.element().getDataValues()); } } diff --git a/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerSubmitTest.java b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerSubmitTest.java index c49bcafe90cd..eb097a9ed293 100644 --- a/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerSubmitTest.java +++ b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/BeamPlannerSubmitTest.java @@ -36,7 +36,7 @@ public void insertSelectFilter() throws Exception { pipeline.run().waitUntilFinish(); Assert.assertTrue(MockedBeamSQLTable.CONTENT.size() == 1); - Assert.assertEquals("order_id=12345,site_id=0,price=20.5", MockedBeamSQLTable.CONTENT.get(0)); + Assert.assertEquals("order_id=12345,site_id=0,price=20.5,order_time=null", MockedBeamSQLTable.CONTENT.get(0)); } } diff --git a/dsls/sql/src/test/java/org/beam/dsls/sql/planner/MockedBeamSQLTable.java b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/MockedBeamSQLTable.java index 5924493f51f7..31f55780c395 100644 --- a/dsls/sql/src/test/java/org/beam/dsls/sql/planner/MockedBeamSQLTable.java +++ b/dsls/sql/src/test/java/org/beam/dsls/sql/planner/MockedBeamSQLTable.java @@ -18,6 +18,7 @@ package org.beam.dsls.sql.planner; import java.util.ArrayList; +import java.util.Date; import java.util.List; import org.apache.beam.sdk.transforms.Create; @@ -60,19 +61,25 @@ public PTransform> buildIOReader() { row1.addField(0, 12345L); row1.addField(1, 0); row1.addField(2, 10.5); - row1.addField(3, System.currentTimeMillis()); + row1.addField(3, new Date()); BeamSQLRow row2 = new BeamSQLRow(beamSqlRecordType); row2.addField(0, 12345L); row2.addField(1, 1); row2.addField(2, 20.5); - row2.addField(3, System.currentTimeMillis()); + row2.addField(3, new Date()); BeamSQLRow row3 = new BeamSQLRow(beamSqlRecordType); row3.addField(0, 12345L); row3.addField(1, 0); row3.addField(2, 20.5); - row3.addField(3, System.currentTimeMillis()); + row3.addField(3, new Date()); + + BeamSQLRow row4 = new BeamSQLRow(beamSqlRecordType); + row4.addField(0, null); + row4.addField(1, null); + row4.addField(2, 20.5); + row4.addField(3, new Date()); return Create.of(row1, row2, row3); } From 4c2fc05dabfad47779ea1f164828f2228abe9994 Mon Sep 17 00:00:00 2001 From: mingmxu Date: Sun, 9 Apr 2017 17:24:02 -0700 Subject: [PATCH 14/15] revert files that're not impacted --- pom.xml | 20 ++++++++++---------- sdks/java/pom.xml | 6 +++--- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/pom.xml b/pom.xml index 02c1ca53a25c..ad9e8cb90181 100644 --- a/pom.xml +++ b/pom.xml @@ -14,7 +14,8 @@ 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. ---> +--> + 4.0.0 @@ -91,13 +92,13 @@ UTF-8 - + true - + 3.5 @@ -151,7 +152,6 @@ sdks/java/build-tools sdks runners - dsls examples sdks/java/javadoc @@ -447,12 +447,12 @@ ${project.version} - + org.apache.beam beam-sdks-java-io-hadoop-input-format - ${project.version} + ${project.version} - + org.apache.beam beam-runners-core-construction-java @@ -1307,7 +1307,7 @@ - + @@ -1321,7 +1321,7 @@ - + @@ -1594,4 +1594,4 @@ 3.2 - \ No newline at end of file + diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index 5180352cfc31..7ca61090e7b2 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -14,7 +14,8 @@ 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. ---> +--> + 4.0.0 @@ -42,7 +43,6 @@ - @@ -57,4 +57,4 @@ - \ No newline at end of file + From 1f55b6d1c6171c3ced46148ef6015a9ae2f85edf Mon Sep 17 00:00:00 2001 From: mingmxu Date: Sun, 9 Apr 2017 17:35:23 -0700 Subject: [PATCH 15/15] code cleanup: reformat imports --- dsls/sql/pom.xml | 4 ---- .../main/java/org/beam/dsls/sql/example/BeamSqlExample.java | 1 - .../org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java | 2 -- .../java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java | 2 -- .../java/org/beam/dsls/sql/planner/BeamPipelineCreator.java | 1 - .../java/org/beam/dsls/sql/planner/BeamQueryPlanner.java | 1 - .../main/java/org/beam/dsls/sql/planner/BeamRuleSets.java | 2 -- .../java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java | 1 - .../main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java | 1 - .../src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java | 2 -- .../main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java | 1 - .../src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java | 1 - .../main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java | 1 - .../main/java/org/beam/dsls/sql/schema/BaseBeamTable.java | 3 +-- .../java/org/beam/dsls/sql/schema/BeamSQLRecordType.java | 3 --- .../org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java | 3 --- .../src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java | 4 ---- .../main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java | 6 ------ .../org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java | 1 - .../java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java | 2 -- .../java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java | 1 - .../java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java | 1 - pom.xml | 2 +- 23 files changed, 2 insertions(+), 44 deletions(-) diff --git a/dsls/sql/pom.xml b/dsls/sql/pom.xml index a4ebf942269a..6819fce23982 100644 --- a/dsls/sql/pom.xml +++ b/dsls/sql/pom.xml @@ -141,10 +141,6 @@ org.slf4j slf4j-api - - org.apache.avro - avro - org.apache.calcite calcite-linq4j diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/example/BeamSqlExample.java b/dsls/sql/src/main/java/org/beam/dsls/sql/example/BeamSqlExample.java index 265c08d01f0a..81ac6eec99cc 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/example/BeamSqlExample.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/example/BeamSqlExample.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; - import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelProtoDataType; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java index 10efcb4ee7b8..48306da0dc91 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/BeamSQLSpELExecutor.java @@ -18,10 +18,8 @@ package org.beam.dsls.sql.interpreter; import static com.google.common.base.Preconditions.checkArgument; - import java.util.ArrayList; import java.util.List; - import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java index 721aaae9d5ba..c7cbace4e360 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/interpreter/CalciteToSpEL.java @@ -18,10 +18,8 @@ package org.beam.dsls.sql.interpreter; import com.google.common.base.Joiner; - import java.util.ArrayList; import java.util.List; - import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java index 826c6a568b1c..5a0c73d2a335 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamPipelineCreator.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.planner; import java.util.Map; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.options.PipelineOptions; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamQueryPlanner.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamQueryPlanner.java index 14656a0e7b30..7a03ae946dcc 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamQueryPlanner.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamQueryPlanner.java @@ -22,7 +22,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.calcite.adapter.java.JavaTypeFactory; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRuleSets.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRuleSets.java index 39c38bb26268..3f40c271c486 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRuleSets.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamRuleSets.java @@ -19,9 +19,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; - import java.util.Iterator; - import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.rel.RelNode; import org.apache.calcite.tools.RuleSet; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java index f5157acc11dd..94b341c82dba 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSQLRelUtils.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.planner; import java.util.concurrent.atomic.AtomicInteger; - import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.plan.volcano.RelSubset; import org.apache.calcite.rel.RelNode; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java index 3cc5a0f8b659..a10b6edfe8ac 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/planner/BeamSqlRunner.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.planner; import java.io.Serializable; - import org.apache.calcite.plan.RelOptUtil; import org.apache.calcite.schema.Schema; import org.apache.calcite.schema.SchemaPlus; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java index 9e8bb0a4fc10..46654e5e372a 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSinkRel.java @@ -18,9 +18,7 @@ package org.beam.dsls.sql.rel; import com.google.common.base.Joiner; - import java.util.List; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java index a7713459cc9e..f14db922e63e 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamIOSourceRel.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.rel; import com.google.common.base.Joiner; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.calcite.plan.RelOptCluster; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java index 8519294e1e70..f4fc2d866afd 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rel/BeamProjectRel.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.rel; import java.util.List; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java index 232269c53ef7..a44c002f05c0 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/rule/BeamIOSinkRule.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.rule; import java.util.List; - import org.apache.calcite.plan.Convention; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BaseBeamTable.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BaseBeamTable.java index 5a815c81ab21..8d31c6def18a 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BaseBeamTable.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BaseBeamTable.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.schema; import java.io.Serializable; - import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -30,9 +29,9 @@ import org.apache.calcite.rel.type.RelProtoDataType; import org.apache.calcite.schema.ScannableTable; import org.apache.calcite.schema.Schema.TableType; -import org.beam.dsls.sql.planner.BeamQueryPlanner; import org.apache.calcite.schema.Statistic; import org.apache.calcite.schema.Statistics; +import org.beam.dsls.sql.planner.BeamQueryPlanner; /** * Each IO in Beam has one table schema, by extending {@link BaseBeamTable}. diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java index d2dd9854285c..dc8e38103cc6 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordType.java @@ -20,9 +20,6 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; - -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.sql.type.SqlTypeName; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java index c30f39e5d8e6..c708c4e50b1c 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRecordTypeCoder.java @@ -1,6 +1,5 @@ package org.beam.dsls.sql.schema; -import com.fasterxml.jackson.annotation.JsonCreator; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -11,7 +10,6 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.beam.sdk.coders.Coder.Context; /** * A {@link Coder} for {@link BeamSQLRecordType}. @@ -24,7 +22,6 @@ public class BeamSQLRecordTypeCoder extends StandardCoder { private static final BeamSQLRecordTypeCoder INSTANCE = new BeamSQLRecordTypeCoder(); private BeamSQLRecordTypeCoder(){} - @JsonCreator public static BeamSQLRecordTypeCoder of() { return INSTANCE; } diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java index d74d84cbeb2a..3ec170e85a71 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSQLRow.java @@ -18,13 +18,9 @@ package org.beam.dsls.sql.schema; import java.io.Serializable; -import java.math.BigInteger; import java.util.ArrayList; import java.util.Date; -import java.util.HashMap; import java.util.List; -import java.util.Map; - import org.apache.calcite.sql.type.SqlTypeName; /** diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java index a93ca25de372..de80dd5a1644 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/BeamSqlRowCoder.java @@ -1,6 +1,5 @@ package org.beam.dsls.sql.schema; -import com.fasterxml.jackson.annotation.JsonCreator; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -8,16 +7,12 @@ import java.util.List; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; -import org.apache.beam.sdk.coders.BigIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.coders.MapCoder; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarIntCoder; -import org.apache.beam.sdk.coders.Coder.Context; /** * A {@link Coder} encodes {@link BeamSQLRow}. @@ -36,7 +31,6 @@ public class BeamSqlRowCoder extends StandardCoder{ private static final BeamSqlRowCoder INSTANCE = new BeamSqlRowCoder(); private BeamSqlRowCoder(){} - @JsonCreator public static BeamSqlRowCoder of() { return INSTANCE; } diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java index 2c5df88bf01e..2570763c3e7b 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaCSVTable.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.schema.kafka; import java.util.List; - import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java index 05178ce02e8a..29f3f927b429 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/schema/kafka/BeamKafkaTable.java @@ -18,11 +18,9 @@ package org.beam.dsls.sql.schema.kafka; import static com.google.common.base.Preconditions.checkArgument; - import java.io.Serializable; import java.util.List; import java.util.Map; - import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.transforms.PTransform; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java index 6a7fb366b68d..06db2802c393 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLFilterFn.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.transform; import java.util.List; - import org.apache.beam.sdk.transforms.DoFn; import org.beam.dsls.sql.interpreter.BeamSQLExpressionExecutor; import org.beam.dsls.sql.rel.BeamFilterRel; diff --git a/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java index 2ad79ccca7c2..12061d2f094c 100644 --- a/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java +++ b/dsls/sql/src/main/java/org/beam/dsls/sql/transform/BeamSQLProjectFn.java @@ -18,7 +18,6 @@ package org.beam.dsls.sql.transform; import java.util.List; - import org.apache.beam.sdk.transforms.DoFn; import org.beam.dsls.sql.interpreter.BeamSQLExpressionExecutor; import org.beam.dsls.sql.rel.BeamProjectRel; diff --git a/pom.xml b/pom.xml index ad9e8cb90181..5749df1fc9ba 100644 --- a/pom.xml +++ b/pom.xml @@ -152,10 +152,10 @@ sdks/java/build-tools sdks runners + dsls examples sdks/java/javadoc - dsls