From 70377ff6db054a610fd3d73177c7fd6839649bdb Mon Sep 17 00:00:00 2001 From: albertshau Date: Fri, 21 Feb 2020 17:40:23 -0800 Subject: [PATCH 1/4] add broadcast join plugin --- pom.xml | 6 +- .../plugin/spark/dynamic/BroadcastJoin.java | 107 ++++++++++++++ .../spark/dynamic/BroadcastJoinConfig.java | 82 +++++++++++ .../plugin/spark/dynamic/BroadcastJoiner.java | 131 ++++++++++++++++++ .../plugin/spark/dynamic/ScalaSparkTest.java | 63 +++++++++ widgets/BroadcastJoin-sparkcompute.json | 75 ++++++++++ widgets/ScalaSparkProgram-sparkprogram.json | 2 +- 7 files changed, 462 insertions(+), 4 deletions(-) create mode 100644 src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java create mode 100644 src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java create mode 100644 src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java create mode 100644 widgets/BroadcastJoin-sparkcompute.json diff --git a/pom.xml b/pom.xml index 9200b0a..c459b8a 100644 --- a/pom.xml +++ b/pom.xml @@ -26,10 +26,10 @@ - system:cdap-data-pipeline[6.0.0-SNAPSHOT,7.0.0-SNAPSHOT) - system:cdap-data-streams[6.0.0-SNAPSHOT,7.0.0-SNAPSHOT) + system:cdap-data-pipeline[6.1.0,7.0.0-SNAPSHOT) + system:cdap-data-streams[6.1.0,7.0.0-SNAPSHOT) - 6.0.0-SNAPSHOT + 6.1.1 2.1.3 1.0.9 diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java new file mode 100644 index 0000000..9b275d1 --- /dev/null +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java @@ -0,0 +1,107 @@ +/* + * Copyright © 2020 Cask Data, Inc. + * + * Licensed 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 io.cdap.plugin.spark.dynamic; + +import io.cdap.cdap.api.annotation.Description; +import io.cdap.cdap.api.annotation.Name; +import io.cdap.cdap.api.annotation.Plugin; +import io.cdap.cdap.api.data.format.StructuredRecord; +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.cdap.etl.api.FailureCollector; +import io.cdap.cdap.etl.api.PipelineConfigurer; +import io.cdap.cdap.etl.api.StageConfigurer; +import io.cdap.cdap.etl.api.batch.SparkCompute; +import io.cdap.cdap.etl.api.batch.SparkExecutionPluginContext; +import org.apache.spark.api.java.JavaRDD; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Performs a broadcast join. + */ +@Plugin(type = SparkCompute.PLUGIN_TYPE) +@Name(BroadcastJoin.NAME) +@Description("Performs a broadcast join by loading a small dataset into memory.") +public class BroadcastJoin extends SparkCompute { + public static final String NAME = "BroadcastJoin"; + private static final Set SUPPORTED_TYPES = + new HashSet<>(Arrays.asList(Schema.Type.INT, Schema.Type.LONG, Schema.Type.FLOAT, Schema.Type.DOUBLE, + Schema.Type.BOOLEAN, Schema.Type.STRING)); + private final BroadcastJoinConfig config; + + public BroadcastJoin(BroadcastJoinConfig config) { + this.config = config; + } + + @Override + public void configurePipeline(PipelineConfigurer pipelineConfigurer) { + StageConfigurer stageConfigurer = pipelineConfigurer.getStageConfigurer(); + Schema inputSchema = stageConfigurer.getInputSchema(); + if (inputSchema == null) { + stageConfigurer.setOutputSchema(null); + return; + } + + FailureCollector failureCollector = stageConfigurer.getFailureCollector(); + // TODO: validation on schema parsing + Schema smallDatasetSchema = config.getSmallDatasetSchema(); + + for (Schema.Field field : smallDatasetSchema.getFields()) { + Schema.Type fieldType = field.getSchema().getNonNullable().getType(); + if (!SUPPORTED_TYPES.contains(fieldType)) { + failureCollector.addFailure(String.format("Field '%s' is an unsupported type", field.getName()), "") + .withConfigElement("schema", field.getName() + fieldType.name().toLowerCase()); + } + } + + List joinKeys = config.getJoinKeys(); + for (String joinKey : joinKeys) { + if (inputSchema.getField(joinKey) == null) { + failureCollector.addFailure("Join key does not exist in the input schema.", + "Select join keys that exists in both the input schema and small dataset schema.") + .withConfigElement("joinOn", joinKey); + } else if (smallDatasetSchema.getField(joinKey) == null) { + failureCollector.addFailure("Join key does not exist in the small dataset schema.", + "Select join keys that exists in both the input schema and small dataset schema.") + .withConfigElement("joinOn", joinKey); + } + } + failureCollector.getOrThrowException(); + + List fields = new ArrayList<>(); + fields.addAll(inputSchema.getFields()); + for (Schema.Field field : smallDatasetSchema.getFields()) { + if (joinKeys.contains(field.getName())) { + continue; + } + fields.add(field); + } + stageConfigurer.setOutputSchema(Schema.recordOf(inputSchema.getRecordName() + ".joined", fields)); + } + + @Override + public JavaRDD transform(SparkExecutionPluginContext context, + JavaRDD javaRDD) { + BroadcastJoiner joiner = new BroadcastJoiner(config); + return joiner.join(context, javaRDD); + } + +} diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java new file mode 100644 index 0000000..e585934 --- /dev/null +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java @@ -0,0 +1,82 @@ +/* + * Copyright © 2020 Cask Data, Inc. + * + * Licensed 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 io.cdap.plugin.spark.dynamic; + +import io.cdap.cdap.api.annotation.Description; +import io.cdap.cdap.api.annotation.Macro; +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.cdap.api.plugin.PluginConfig; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +/** + * Config for broadcast join. + */ +public class BroadcastJoinConfig extends PluginConfig { + @Macro + @Description("Path to load the small dataset from") + private String path; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter; + + @Description("Schema of small dataset") + private String datasetSchema; + + @Description("Keys to join on") + private String joinOn; + + @Nullable + @Description("Type of join") + private String joinType; + + public String getPath() { + return path; + } + + public String getDelimiter() { + return delimiter == null ? "," : delimiter; + } + + public String getJoinType() { + return joinType == null ? "INNER" : joinType; + } + + public Schema getSmallDatasetSchema() { + List fields = new ArrayList<>(); + // key1 type1, key2 type2, ... + for (String keyType : datasetSchema.split(",")) { + keyType = keyType.trim(); + int idx = keyType.lastIndexOf(" "); + String fieldName = keyType.substring(0, idx); + String typeStr = keyType.substring(idx + 1); + Schema.Type type = Schema.Type.valueOf(typeStr.toUpperCase()); + fields.add(Schema.Field.of(fieldName, Schema.nullableOf(Schema.of(type)))); + } + return Schema.recordOf("smallDataset", fields); + } + + public List getJoinKeys() { + return Arrays.stream(joinOn.split(",")).collect(Collectors.toList()); + } +} diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java new file mode 100644 index 0000000..2d5e1aa --- /dev/null +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java @@ -0,0 +1,131 @@ +/* + * Copyright © 2020 Cask Data, Inc. + * + * Licensed 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 io.cdap.plugin.spark.dynamic; + +import io.cdap.cdap.api.data.format.StructuredRecord; +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.cdap.api.spark.sql.DataFrames; +import io.cdap.cdap.etl.api.batch.SparkExecutionPluginContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.StructType; +import scala.collection.JavaConverters; +import scala.collection.Seq; + +import java.io.Serializable; +import java.util.Iterator; +import java.util.regex.Pattern; + +/** + * Performs the join. Separate from the plugin to allow validation endpoint to be free of spark classes. + */ +public class BroadcastJoiner implements Serializable { + private final BroadcastJoinConfig config; + + public BroadcastJoiner(BroadcastJoinConfig config) { + this.config = config; + } + + public JavaRDD join(SparkExecutionPluginContext context, + JavaRDD javaRDD) { + SQLContext sqlContext = new SQLContext(context.getSparkContext()); + Schema inputSchema = context.getInputSchema(); + if (inputSchema == null) { + inputSchema = javaRDD.take(1).get(0).getSchema(); + } + + StructType inputSparkSchema = DataFrames.toDataType(inputSchema); + Dataset inputDataset = sqlContext.createDataFrame(javaRDD.map(r -> DataFrames.toRow(r, inputSparkSchema)), + inputSparkSchema); + + Schema smallDatasetSchema = config.getSmallDatasetSchema(); + + // TODO: replace with .hadoopFile() + Dataset smallDataset = sqlContext.createDataFrame( + context.getSparkContext().textFile(config.getPath()) + .map(line -> { + /* + Parse ourselves, can't do: + + Dataset smallDataset = sqlContext.read() + .option("delimiter", config.getDelimiter()) + .schema(DataFrames.toDataType(smallDatasetSchema)) + .csv(config.path); + + because multi-character delimiters are not supported + */ + + String[] lineFields = line.split(Pattern.quote(config.getDelimiter())); + Object[] vals = new Object[lineFields.length]; + Iterator fieldIterator = smallDatasetSchema.getFields().iterator(); + for (int i = 0; i < vals.length; i++) { + String fieldStr = lineFields[i]; + + if (!fieldIterator.hasNext()) { + vals[i] = null; + continue; + } + + Schema fieldSchema = fieldIterator.next().getSchema(); + + if (fieldStr == null || fieldStr.isEmpty()) { + vals[i] = null; + continue; + } + + Schema.Type fieldType = fieldSchema.getNonNullable().getType(); + switch (fieldType) { + case STRING: + vals[i] = fieldStr; + break; + case INT: + vals[i] = Integer.parseInt(fieldStr); + break; + case LONG: + vals[i] = Long.parseLong(fieldStr); + break; + case FLOAT: + vals[i] = Float.parseFloat(fieldStr); + break; + case DOUBLE: + vals[i] = Double.parseDouble(fieldStr); + break; + case BOOLEAN: + vals[i] = Boolean.parseBoolean(fieldStr); + break; + default: + // should never happen, as it should be checked at configure time + } + } + return RowFactory.create(vals); + }), + DataFrames.toDataType(smallDatasetSchema)); + + + Seq joinKeys = JavaConverters.collectionAsScalaIterableConverter(config.getJoinKeys()).asScala().toSeq(); + Dataset joined = inputDataset.join(functions.broadcast(smallDataset), + joinKeys, config.getJoinType()); + + Schema outputSchema = DataFrames.toSchema(joined.schema()); + return joined.javaRDD().map(row -> DataFrames.fromRow(row, outputSchema)); + } + +} diff --git a/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java b/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java index 168fef8..d6c7e91 100644 --- a/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java +++ b/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java @@ -64,10 +64,12 @@ import java.io.BufferedReader; import java.io.File; import java.io.FileReader; +import java.io.FileWriter; import java.io.OutputStream; import java.io.PrintWriter; import java.io.StringWriter; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -406,6 +408,67 @@ public void testScalaSparkSinkDataFrame() throws Exception { testWordCountSink(codeWriter.toString(), outputFolder); } + @Test + public void testBroadcastJoin() throws Exception { + File smallDataset = TEMP_FOLDER.newFile("smallDataset.csv"); + + try (FileWriter fileWriter = new FileWriter(smallDataset)) { + fileWriter.write("0|;|alice\n"); + fileWriter.write("1|;|bob\n"); + } + + Schema schema = Schema.recordOf("purchases", + Schema.Field.of("userid", Schema.of(Schema.Type.INT)), + Schema.Field.of("itemid", Schema.of(Schema.Type.LONG)), + Schema.Field.of("itemname", Schema.of(Schema.Type.STRING)), + Schema.Field.of("price", Schema.of(Schema.Type.DOUBLE))); + + Map properties = new HashMap<>(); + properties.put("path", smallDataset.getAbsolutePath()); + properties.put("datasetSchema", "userid int, username string"); + properties.put("delimiter", "|;|"); + properties.put("joinOn", "userid"); + String inputTableName = UUID.randomUUID().toString(); + String outputTableName = UUID.randomUUID().toString(); + ETLBatchConfig config = ETLBatchConfig.builder() + .addStage(new ETLStage("src", MockSource.getPlugin(inputTableName, schema))) + .addStage(new ETLStage("join", new ETLPlugin(BroadcastJoin.NAME, SparkCompute.PLUGIN_TYPE, properties))) + .addStage(new ETLStage("sink", MockSink.getPlugin(outputTableName))) + .addConnection("src", "join") + .addConnection("join", "sink") + .build(); + + ArtifactSummary artifactSummary = new ArtifactSummary(DATAPIPELINE_ARTIFACT_ID.getArtifact(), + DATAPIPELINE_ARTIFACT_ID.getVersion()); + AppRequest appRequest = new AppRequest<>(artifactSummary, config); + ApplicationId appId = NamespaceId.DEFAULT.app("testBroadcastJoin"); + ApplicationManager appManager = deployApplication(appId, appRequest); + + // setup input data + StructuredRecord record = StructuredRecord.builder(schema) + .set("userid", 0) + .set("itemid", 1000L) + .set("itemname", "donut") + .set("price", 1.0d) + .build(); + List inputRecords = Arrays.asList(record); + DataSetManager inputTableManager = getDataset(inputTableName); + MockSource.writeInput(inputTableManager, inputRecords); + + WorkflowManager workflowManager = appManager.getWorkflowManager(SmartWorkflow.NAME); + workflowManager.startAndWaitForRun(ProgramRunStatus.COMPLETED, 3, TimeUnit.MINUTES); + + DataSetManager
outputTableManager = getDataset(outputTableName); + List output = MockSink.readOutput(outputTableManager); + Assert.assertEquals(1, output.size()); + StructuredRecord joined = output.iterator().next(); + Assert.assertEquals(0, (int) joined.get("userid")); + Assert.assertEquals("donut", joined.get("itemname")); + Assert.assertEquals(1.0d, (double) joined.get("price"), 0.000001); + Assert.assertEquals(1000L, (long) joined.get("itemid")); + Assert.assertEquals("alice", joined.get("username")); + } + private void testWordCountSink(String code, File outputFolder) throws Exception { Schema inputSchema = Schema.recordOf( "input", diff --git a/widgets/BroadcastJoin-sparkcompute.json b/widgets/BroadcastJoin-sparkcompute.json new file mode 100644 index 0000000..8412e48 --- /dev/null +++ b/widgets/BroadcastJoin-sparkcompute.json @@ -0,0 +1,75 @@ +{ + "metadata": { + "spec-version": "1.4" + }, + "configuration-groups": [ + { + "label": "Basic", + "properties": [ + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter", + "widget-attributes": { + "default": "," + } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn" + }, + { + "widget-type": "select", + "label": "Join Type", + "name": "joinType", + "widget-attributes": { + "values": [ + "inner", + "outer", + "leftouter", + "rightouter" + ], + "default": "inner" + } + }, + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" + ] + }, + "plugin-function": { + "label": "Get Schema", + "widget": "outputSchema", + "output-property": "schema", + "omit-properties": [ + { + "name": "schema" + } + ] + } + } + ] + } + ], + "outputs": [ ] +} \ No newline at end of file diff --git a/widgets/ScalaSparkProgram-sparkprogram.json b/widgets/ScalaSparkProgram-sparkprogram.json index 6257c35..2fdf10d 100644 --- a/widgets/ScalaSparkProgram-sparkprogram.json +++ b/widgets/ScalaSparkProgram-sparkprogram.json @@ -19,7 +19,7 @@ "label": "Scala", "name": "scalaCode", "widget-attributes": { - "default": "import co.cask.cdap.api.spark._\nimport org.apache.spark._\nimport org.slf4j._\n\nclass SparkProgram extends SparkMain {\n import SparkProgram._\n\n override def run(implicit sec: SparkExecutionContext): Unit = {\n LOG.info(\"Spark Program Started\")\n\n val sc = new SparkContext\n\n LOG.info(\"Spark Program Completed\")\n }\n}\n\nobject SparkProgram {\n val LOG = LoggerFactory.getLogger(getClass())\n}" + "default": "import io.cdap.cdap.api.spark._\nimport org.apache.spark._\nimport org.slf4j._\n\nclass SparkProgram extends SparkMain {\n import SparkProgram._\n\n override def run(implicit sec: SparkExecutionContext): Unit = {\n LOG.info(\"Spark Program Started\")\n\n val sc = new SparkContext\n\n LOG.info(\"Spark Program Completed\")\n }\n}\n\nobject SparkProgram {\n val LOG = LoggerFactory.getLogger(getClass())\n}" } }, { From 98b9f6a10281f544be2785ecfe15a85a8ddf00f0 Mon Sep 17 00:00:00 2001 From: albertshau Date: Fri, 28 Feb 2020 11:03:19 -0800 Subject: [PATCH 2/4] add a bunch more datasets to join --- .../plugin/spark/dynamic/BroadcastJoin.java | 49 +- .../spark/dynamic/BroadcastJoinConfig.java | 242 ++++++- .../plugin/spark/dynamic/BroadcastJoiner.java | 155 +++-- .../plugin/spark/dynamic/DatasetJoinInfo.java | 66 ++ .../plugin/spark/dynamic/ScalaSparkTest.java | 16 +- widgets/BroadcastJoin-sparkcompute.json | 591 +++++++++++++++++- 6 files changed, 992 insertions(+), 127 deletions(-) create mode 100644 src/main/java/io/cdap/plugin/spark/dynamic/DatasetJoinInfo.java diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java index 9b275d1..a151028 100644 --- a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java @@ -61,40 +61,35 @@ public void configurePipeline(PipelineConfigurer pipelineConfigurer) { } FailureCollector failureCollector = stageConfigurer.getFailureCollector(); - // TODO: validation on schema parsing - Schema smallDatasetSchema = config.getSmallDatasetSchema(); - - for (Schema.Field field : smallDatasetSchema.getFields()) { - Schema.Type fieldType = field.getSchema().getNonNullable().getType(); - if (!SUPPORTED_TYPES.contains(fieldType)) { - failureCollector.addFailure(String.format("Field '%s' is an unsupported type", field.getName()), "") - .withConfigElement("schema", field.getName() + fieldType.name().toLowerCase()); - } - } - - List joinKeys = config.getJoinKeys(); - for (String joinKey : joinKeys) { - if (inputSchema.getField(joinKey) == null) { - failureCollector.addFailure("Join key does not exist in the input schema.", - "Select join keys that exists in both the input schema and small dataset schema.") - .withConfigElement("joinOn", joinKey); - } else if (smallDatasetSchema.getField(joinKey) == null) { - failureCollector.addFailure("Join key does not exist in the small dataset schema.", - "Select join keys that exists in both the input schema and small dataset schema.") - .withConfigElement("joinOn", joinKey); + List datasetJoins = config.getDatasetsToJoin(); + Schema schema = inputSchema; + for (DatasetJoinInfo joinInfo : datasetJoins) { + for (String joinKey : joinInfo.getJoinKeys()) { + if (schema.getField(joinKey) == null) { + failureCollector.addFailure("Join key does not exist in the schema.", "") + .withConfigElement("joinOn" + joinInfo.getNum(), joinKey); + } } + schema = joinSchemas(schema, joinInfo.getSchema(), joinInfo.getJoinKeys()); } - failureCollector.getOrThrowException(); + stageConfigurer.setOutputSchema(schema); + } - List fields = new ArrayList<>(); - fields.addAll(inputSchema.getFields()); - for (Schema.Field field : smallDatasetSchema.getFields()) { + private Schema joinSchemas(Schema schema1, Schema schema2, Set joinKeys) { + List fields = + new ArrayList<>(schema1.getFields().size() + schema2.getFields().size() - joinKeys.size()); + fields.addAll(schema1.getFields()); + for (Schema.Field field : schema2.getFields()) { if (joinKeys.contains(field.getName())) { continue; } - fields.add(field); + if (field.getSchema().isNullable()) { + fields.add(field); + } else { + fields.add(Schema.Field.of(field.getName(), Schema.nullableOf(field.getSchema()))); + } } - stageConfigurer.setOutputSchema(Schema.recordOf(inputSchema.getRecordName() + ".joined", fields)); + return Schema.recordOf("joined", fields); } @Override diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java index e585934..a58e03a 100644 --- a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java @@ -23,7 +23,10 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import javax.annotation.Nullable; @@ -31,6 +34,11 @@ * Config for broadcast join. */ public class BroadcastJoinConfig extends PluginConfig { + + @Macro + @Description("Number of datasets to join") + private Integer numJoins; + @Macro @Description("Path to load the small dataset from") private String path; @@ -50,22 +58,230 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Type of join") private String joinType; - public String getPath() { - return path; - } + /* + Hacks... + + Dataset 2 + */ + + @Macro + @Nullable + @Description("Path to load the small dataset from") + private String path2; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter2; + + @Nullable + @Description("Schema of small dataset") + private String datasetSchema2; + + @Nullable + @Description("Keys to join on") + private String joinOn2; + + @Nullable + @Description("Type of join") + private String joinType2; + + /* + Dataset 3 + */ + + @Macro + @Nullable + @Description("Path to load the small dataset from") + private String path3; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter3; + + @Nullable + @Description("Schema of small dataset") + private String datasetSchema3; + + @Nullable + @Description("Keys to join on") + private String joinOn3; + + @Nullable + @Description("Type of join") + private String joinType3; + + /* + Dataset 4 + */ + + @Macro + @Nullable + @Description("Path to load the small dataset from") + private String path4; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter4; + + @Nullable + @Description("Schema of small dataset") + private String datasetSchema4; + + @Nullable + @Description("Keys to join on") + private String joinOn4; + + @Nullable + @Description("Type of join") + private String joinType4; + + /* + Dataset 5 + */ + + @Macro + @Nullable + @Description("Path to load the small dataset from") + private String path5; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter5; + + @Nullable + @Description("Schema of small dataset") + private String datasetSchema5; - public String getDelimiter() { - return delimiter == null ? "," : delimiter; + @Nullable + @Description("Keys to join on") + private String joinOn5; + + @Nullable + @Description("Type of join") + private String joinType5; + + /* + Dataset 6 + */ + + @Macro + @Nullable + @Description("Path to load the small dataset from") + private String path6; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter6; + + @Nullable + @Description("Schema of small dataset") + private String datasetSchema6; + + @Nullable + @Description("Keys to join on") + private String joinOn6; + + @Nullable + @Description("Type of join") + private String joinType6; + + /* + Dataset 7 + */ + + @Macro + @Nullable + @Description("Path to load the small dataset from") + private String path7; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter7; + + @Nullable + @Description("Schema of small dataset") + private String datasetSchema7; + + @Nullable + @Description("Keys to join on") + private String joinOn7; + + @Nullable + @Description("Type of join") + private String joinType7; + + /* + Dataset 8 + */ + + @Macro + @Nullable + @Description("Path to load the small dataset from") + private String path8; + + @Macro + @Nullable + @Description("Delimiter used in the small dataset") + private String delimiter8; + + @Nullable + @Description("Schema of small dataset") + private String datasetSchema8; + + @Nullable + @Description("Keys to join on") + private String joinOn8; + + @Nullable + @Description("Type of join") + private String joinType8; + + public List getDatasetsToJoin() { + List datasetJoinInfos = new ArrayList<>(8); + for (int i = 0; i < numJoins; i++) { + datasetJoinInfos.add(getJoinInfo(i + 1)); + } + return datasetJoinInfos; } - public String getJoinType() { - return joinType == null ? "INNER" : joinType; + private DatasetJoinInfo getJoinInfo(int datasetNum) { + Map rawProperties = getProperties().getProperties(); + String path = rawProperties.get(getPropertyName("path", datasetNum)); + if (path == null) { + throw new IllegalArgumentException("Path for Dataset " + datasetNum + " must be specified."); + } + + String joinType = rawProperties.get(getPropertyName("joinType", datasetNum)); + joinType = joinType == null ? "INNER" : joinType.toUpperCase(); + String delimiter = rawProperties.get(getPropertyName("delimiter", datasetNum)); + delimiter = delimiter == null ? "," : delimiter; + + String schemaStr = rawProperties.get(getPropertyName("datasetSchema", datasetNum)); + if (schemaStr == null || schemaStr.isEmpty()) { + throw new IllegalArgumentException("Schema for Dataset " + datasetNum + " must be specified."); + } + Schema schema = parseSchema(schemaStr); + + String joinOnStr = rawProperties.get(getPropertyName("joinOn", datasetNum)); + if (joinOnStr == null || joinOnStr.isEmpty()) { + throw new IllegalArgumentException("Join keys for Dataset " + datasetNum + " must be specified."); + } + Set joinKeys = parseJoinKeys(joinOnStr); + + return new DatasetJoinInfo(datasetNum, path, joinType, delimiter, joinKeys, schema); } - public Schema getSmallDatasetSchema() { + // schema of form: name1 type1, name2 type2, ... + private Schema parseSchema(String schemaStr) { List fields = new ArrayList<>(); // key1 type1, key2 type2, ... - for (String keyType : datasetSchema.split(",")) { + for (String keyType : schemaStr.split(",")) { keyType = keyType.trim(); int idx = keyType.lastIndexOf(" "); String fieldName = keyType.substring(0, idx); @@ -76,7 +292,11 @@ public Schema getSmallDatasetSchema() { return Schema.recordOf("smallDataset", fields); } - public List getJoinKeys() { - return Arrays.stream(joinOn.split(",")).collect(Collectors.toList()); + private Set parseJoinKeys(String joinOn) { + return Arrays.stream(joinOn.split(",")).collect(Collectors.toSet()); + } + + private String getPropertyName(String baseName, int num) { + return num == 1 ? baseName : baseName + num; } } diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java index 2d5e1aa..121390a 100644 --- a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoiner.java @@ -21,6 +21,8 @@ import io.cdap.cdap.api.spark.sql.DataFrames; import io.cdap.cdap.etl.api.batch.SparkExecutionPluginContext; import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; @@ -46,7 +48,8 @@ public BroadcastJoiner(BroadcastJoinConfig config) { public JavaRDD join(SparkExecutionPluginContext context, JavaRDD javaRDD) { - SQLContext sqlContext = new SQLContext(context.getSparkContext()); + JavaSparkContext sparkContext = context.getSparkContext(); + SQLContext sqlContext = new SQLContext(sparkContext); Schema inputSchema = context.getInputSchema(); if (inputSchema == null) { inputSchema = javaRDD.take(1).get(0).getSchema(); @@ -56,76 +59,92 @@ public JavaRDD join(SparkExecutionPluginContext context, Dataset inputDataset = sqlContext.createDataFrame(javaRDD.map(r -> DataFrames.toRow(r, inputSparkSchema)), inputSparkSchema); - Schema smallDatasetSchema = config.getSmallDatasetSchema(); - - // TODO: replace with .hadoopFile() - Dataset smallDataset = sqlContext.createDataFrame( - context.getSparkContext().textFile(config.getPath()) - .map(line -> { - /* - Parse ourselves, can't do: - - Dataset smallDataset = sqlContext.read() - .option("delimiter", config.getDelimiter()) - .schema(DataFrames.toDataType(smallDatasetSchema)) - .csv(config.path); - - because multi-character delimiters are not supported - */ - - String[] lineFields = line.split(Pattern.quote(config.getDelimiter())); - Object[] vals = new Object[lineFields.length]; - Iterator fieldIterator = smallDatasetSchema.getFields().iterator(); - for (int i = 0; i < vals.length; i++) { - String fieldStr = lineFields[i]; - - if (!fieldIterator.hasNext()) { - vals[i] = null; - continue; - } - - Schema fieldSchema = fieldIterator.next().getSchema(); - - if (fieldStr == null || fieldStr.isEmpty()) { - vals[i] = null; - continue; - } - - Schema.Type fieldType = fieldSchema.getNonNullable().getType(); - switch (fieldType) { - case STRING: - vals[i] = fieldStr; - break; - case INT: - vals[i] = Integer.parseInt(fieldStr); - break; - case LONG: - vals[i] = Long.parseLong(fieldStr); - break; - case FLOAT: - vals[i] = Float.parseFloat(fieldStr); - break; - case DOUBLE: - vals[i] = Double.parseDouble(fieldStr); - break; - case BOOLEAN: - vals[i] = Boolean.parseBoolean(fieldStr); - break; - default: - // should never happen, as it should be checked at configure time - } - } - return RowFactory.create(vals); - }), - DataFrames.toDataType(smallDatasetSchema)); - - - Seq joinKeys = JavaConverters.collectionAsScalaIterableConverter(config.getJoinKeys()).asScala().toSeq(); - Dataset joined = inputDataset.join(functions.broadcast(smallDataset), - joinKeys, config.getJoinType()); + Dataset joined = inputDataset; + for (DatasetJoinInfo joinInfo : config.getDatasetsToJoin()) { + Schema schema = joinInfo.getSchema(); + // TODO: replace with .hadoopFile() + Dataset smallDataset = sqlContext.createDataFrame( + sparkContext.textFile(joinInfo.getPath()).map(new CSVParseFunction(joinInfo.getDelimiter(), schema)), + DataFrames.toDataType(schema)); + + Seq joinKeys = JavaConverters.collectionAsScalaIterableConverter(joinInfo.getJoinKeys()) + .asScala() + .toSeq(); + joined = joined.join(functions.broadcast(smallDataset), joinKeys, joinInfo.getJoinType()); + } Schema outputSchema = DataFrames.toSchema(joined.schema()); return joined.javaRDD().map(row -> DataFrames.fromRow(row, outputSchema)); } + /** + * parses csv + */ + public static class CSVParseFunction implements Function { + private final String delimiter; + private final Schema schema; + + public CSVParseFunction(String delimiter, Schema schema) { + this.delimiter = Pattern.quote(delimiter); + this.schema = schema; + } + + @Override + public Row call(String line) { + /* + Parse ourselves, can't do: + + Dataset smallDataset = sqlContext.read() + .option("delimiter", config.getDelimiter()) + .schema(DataFrames.toDataType(smallDatasetSchema)) + .csv(config.path); + + because multi-character delimiters are not supported + */ + + String[] lineFields = line.split(delimiter); + Object[] vals = new Object[lineFields.length]; + Iterator fieldIterator = schema.getFields().iterator(); + for (int i = 0; i < vals.length; i++) { + String fieldStr = lineFields[i]; + + if (!fieldIterator.hasNext()) { + vals[i] = null; + continue; + } + + Schema fieldSchema = fieldIterator.next().getSchema(); + + if (fieldStr == null || fieldStr.isEmpty()) { + vals[i] = null; + continue; + } + + Schema.Type fieldType = fieldSchema.getNonNullable().getType(); + switch (fieldType) { + case STRING: + vals[i] = fieldStr; + break; + case INT: + vals[i] = Integer.parseInt(fieldStr); + break; + case LONG: + vals[i] = Long.parseLong(fieldStr); + break; + case FLOAT: + vals[i] = Float.parseFloat(fieldStr); + break; + case DOUBLE: + vals[i] = Double.parseDouble(fieldStr); + break; + case BOOLEAN: + vals[i] = Boolean.parseBoolean(fieldStr); + break; + default: + // should never happen, as it should be checked at configure time + } + } + return RowFactory.create(vals); + } + } } diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/DatasetJoinInfo.java b/src/main/java/io/cdap/plugin/spark/dynamic/DatasetJoinInfo.java new file mode 100644 index 0000000..7d7ce9f --- /dev/null +++ b/src/main/java/io/cdap/plugin/spark/dynamic/DatasetJoinInfo.java @@ -0,0 +1,66 @@ +/* + * Copyright © 2020 Cask Data, Inc. + * + * Licensed 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 io.cdap.plugin.spark.dynamic; + +import io.cdap.cdap.api.data.schema.Schema; + +import java.util.Set; + +/** + * Information about a single dataset join + */ +public class DatasetJoinInfo { + private final int num; + private final String path; + private final String joinType; + private final String delimiter; + private final Set joinOn; + private final Schema schema; + + public DatasetJoinInfo(int num, String path, String joinType, String delimiter, Set joinOn, Schema schema) { + this.num = num; + this.path = path; + this.joinType = joinType; + this.delimiter = delimiter; + this.joinOn = joinOn; + this.schema = schema; + } + + public int getNum() { + return num; + } + + public String getPath() { + return path; + } + + public String getJoinType() { + return joinType; + } + + public String getDelimiter() { + return delimiter; + } + + public Set getJoinKeys() { + return joinOn; + } + + public Schema getSchema() { + return schema; + } +} diff --git a/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java b/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java index d6c7e91..740e04b 100644 --- a/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java +++ b/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java @@ -410,12 +410,17 @@ public void testScalaSparkSinkDataFrame() throws Exception { @Test public void testBroadcastJoin() throws Exception { - File smallDataset = TEMP_FOLDER.newFile("smallDataset.csv"); + File smallDataset1 = TEMP_FOLDER.newFile("smallDataset1.csv"); + File smallDataset2 = TEMP_FOLDER.newFile("smallDataset2.csv"); - try (FileWriter fileWriter = new FileWriter(smallDataset)) { + try (FileWriter fileWriter = new FileWriter(smallDataset1)) { fileWriter.write("0|;|alice\n"); fileWriter.write("1|;|bob\n"); } + try (FileWriter fileWriter = new FileWriter(smallDataset2)) { + fileWriter.write("alice,morgan\n"); + fileWriter.write("bob,vance\n"); + } Schema schema = Schema.recordOf("purchases", Schema.Field.of("userid", Schema.of(Schema.Type.INT)), @@ -424,10 +429,14 @@ public void testBroadcastJoin() throws Exception { Schema.Field.of("price", Schema.of(Schema.Type.DOUBLE))); Map properties = new HashMap<>(); - properties.put("path", smallDataset.getAbsolutePath()); + properties.put("numJoins", "2"); + properties.put("path", smallDataset1.getAbsolutePath()); properties.put("datasetSchema", "userid int, username string"); properties.put("delimiter", "|;|"); properties.put("joinOn", "userid"); + properties.put("path2", smallDataset2.getAbsolutePath()); + properties.put("datasetSchema2", "username string, lastname string"); + properties.put("joinOn2", "username"); String inputTableName = UUID.randomUUID().toString(); String outputTableName = UUID.randomUUID().toString(); ETLBatchConfig config = ETLBatchConfig.builder() @@ -467,6 +476,7 @@ public void testBroadcastJoin() throws Exception { Assert.assertEquals(1.0d, (double) joined.get("price"), 0.000001); Assert.assertEquals(1000L, (long) joined.get("itemid")); Assert.assertEquals("alice", joined.get("username")); + Assert.assertEquals("morgan", joined.get("lastname")); } private void testWordCountSink(String code, File outputFolder) throws Exception { diff --git a/widgets/BroadcastJoin-sparkcompute.json b/widgets/BroadcastJoin-sparkcompute.json index 8412e48..a9c8fcf 100644 --- a/widgets/BroadcastJoin-sparkcompute.json +++ b/widgets/BroadcastJoin-sparkcompute.json @@ -6,37 +6,167 @@ { "label": "Basic", "properties": [ + { + "widget-type": "select", + "label": "Number of Joins", + "name": "numJoins", + "widget-attributes": { + "values": [ + "1", + "2", + "3", + "4", + "5", + "6", + "7", + "8" + ], + "default": "1" + } + } + ] + }, + { + "label": "Dataset 1", + "properties": [ + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" + ] + } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn" + }, + { + "widget-type": "radio-group", + "label": "Join Type", + "name": "joinType", + "widget-attributes": { + "layout": "inline", + "options": [ + { + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" + } + ], + "default": "inner" + } + }, { "widget-type": "textbox", "label": "Delimiter", "name": "delimiter", "widget-attributes": { "default": "," + }, + "plugin-function": { + "label": "Get Schema", + "widget": "outputSchema", + "output-property": "schema", + "omit-properties": [ + { + "name": "schema" + } + ] + } + } + ] + }, + { + "label": "Dataset 2", + "properties": [ + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path2", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema2", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" + ] } }, { "widget-type": "csv", "label": "Join Keys", - "name": "joinOn" + "name": "joinOn2" }, { - "widget-type": "select", + "widget-type": "radio-group", "label": "Join Type", - "name": "joinType", + "name": "joinType2", "widget-attributes": { - "values": [ - "inner", - "outer", - "leftouter", - "rightouter" + "layout": "inline", + "options": [ + { + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" + } ], "default": "inner" } }, + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter2", + "widget-attributes": { + "default": "," + } + } + ] + }, + { + "label": "Dataset 3", + "properties": [ { "widget-type": "textbox", "label": "Path to Lookup Data", - "name": "path", + "name": "path3", "widget-attributes": { "placeholder": "gs://bucket/path/to/file" } @@ -44,7 +174,7 @@ { "widget-type": "keyvalue-dropdown", "label": "Lookup Data Schema", - "name": "datasetSchema", + "name": "datasetSchema3", "widget-attributes": { "delimiter": ",", "kv-delimiter": " ", @@ -56,20 +186,445 @@ "float", "double" ] - }, - "plugin-function": { - "label": "Get Schema", - "widget": "outputSchema", - "output-property": "schema", - "omit-properties": [ + } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn3" + }, + { + "widget-type": "radio-group", + "label": "Join Type", + "name": "joinType3", + "widget-attributes": { + "layout": "inline", + "options": [ { - "name": "schema" + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" } + ], + "default": "inner" + } + }, + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter3", + "widget-attributes": { + "default": "," + } + } + ] + }, + { + "label": "Dataset 4", + "properties": [ + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path4", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema4", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" ] } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn4" + }, + { + "widget-type": "radio-group", + "label": "Join Type", + "name": "joinType4", + "widget-attributes": { + "layout": "inline", + "options": [ + { + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" + } + ], + "default": "inner" + } + }, + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter4", + "widget-attributes": { + "default": "," + } + } + ] + }, + { + "label": "Dataset 5", + "properties": [ + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path5", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema5", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" + ] + } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn5" + }, + { + "widget-type": "radio-group", + "label": "Join Type", + "name": "joinType5", + "widget-attributes": { + "layout": "inline", + "options": [ + { + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" + } + ], + "default": "inner" + } + }, + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter5", + "widget-attributes": { + "default": "," + } + } + ] + }, + { + "label": "Dataset 6", + "properties": [ + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path6", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema6", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" + ] + } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn6" + }, + { + "widget-type": "radio-group", + "label": "Join Type", + "name": "joinType6", + "widget-attributes": { + "layout": "inline", + "options": [ + { + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" + } + ], + "default": "inner" + } + }, + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter6", + "widget-attributes": { + "default": "," + } + } + ] + }, + { + "label": "Dataset 7", + "properties": [ + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path7", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema7", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" + ] + } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn7" + }, + { + "widget-type": "radio-group", + "label": "Join Type", + "name": "joinType7", + "widget-attributes": { + "layout": "inline", + "options": [ + { + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" + } + ], + "default": "inner" + } + }, + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter7", + "widget-attributes": { + "default": "," + } + } + ] + }, + { + "label": "Dataset 8", + "properties": [ + { + "widget-type": "textbox", + "label": "Path to Lookup Data", + "name": "path8", + "widget-attributes": { + "placeholder": "gs://bucket/path/to/file" + } + }, + { + "widget-type": "keyvalue-dropdown", + "label": "Lookup Data Schema", + "name": "datasetSchema8", + "widget-attributes": { + "delimiter": ",", + "kv-delimiter": " ", + "dropdownOptions": [ + "string", + "boolean", + "int", + "long", + "float", + "double" + ] + } + }, + { + "widget-type": "csv", + "label": "Join Keys", + "name": "joinOn8" + }, + { + "widget-type": "radio-group", + "label": "Join Type", + "name": "joinType8", + "widget-attributes": { + "layout": "inline", + "options": [ + { + "id": "inner", + "label": "Inner" + }, + { + "id": "leftouter", + "label": "Left Outer" + } + ], + "default": "inner" + } + }, + { + "widget-type": "textbox", + "label": "Delimiter", + "name": "delimiter8", + "widget-attributes": { + "default": "," + } } ] } ], - "outputs": [ ] + "outputs": [ ], + "filters": [ + { + "name": "Dataset2", + "condition": { + "expression": "numJoins > 1" + }, + "show": [ + { "name": "delimiter2" }, + { "name": "joinOn2" }, + { "name": "joinType2" }, + { "name": "path2" }, + { "name": "datasetSchema2" } + ] + }, + { + "name": "Dataset3", + "condition": { + "expression": "numJoins > 2" + }, + "show": [ + { "name": "delimiter3" }, + { "name": "joinOn3" }, + { "name": "joinType3" }, + { "name": "path3" }, + { "name": "datasetSchema3" } + ] + }, + { + "name": "Dataset4", + "condition": { + "expression": "numJoins > 3" + }, + "show": [ + { "name": "delimiter4" }, + { "name": "joinOn4" }, + { "name": "joinType4" }, + { "name": "path4" }, + { "name": "datasetSchema4" } + ] + }, + { + "name": "Dataset5", + "condition": { + "expression": "numJoins > 4" + }, + "show": [ + { "name": "delimiter5" }, + { "name": "joinOn5" }, + { "name": "joinType5" }, + { "name": "path5" }, + { "name": "datasetSchema5" } + ] + }, + { + "name": "Dataset6", + "condition": { + "expression": "numJoins > 5" + }, + "show": [ + { "name": "delimiter6" }, + { "name": "joinOn6" }, + { "name": "joinType6" }, + { "name": "path6" }, + { "name": "datasetSchema6" } + ] + }, + { + "name": "Dataset7", + "condition": { + "expression": "numJoins > 6" + }, + "show": [ + { "name": "delimiter7" }, + { "name": "joinOn7" }, + { "name": "joinType7" }, + { "name": "path7" }, + { "name": "datasetSchema7" } + ] + }, + { + "name": "Dataset8", + "condition": { + "expression": "numJoins > 7" + }, + "show": [ + { "name": "delimiter8" }, + { "name": "joinOn8" }, + { "name": "joinType8" }, + { "name": "path8" }, + { "name": "datasetSchema8" } + ] + } + ] } \ No newline at end of file From e723684cfe658051bec27019fac18d575c4555b4 Mon Sep 17 00:00:00 2001 From: albertshau Date: Mon, 2 Mar 2020 14:46:53 -0800 Subject: [PATCH 3/4] fix macros with broadcast join --- .../plugin/spark/dynamic/BroadcastJoin.java | 2 +- .../spark/dynamic/BroadcastJoinConfig.java | 39 +++++++++ .../plugin/spark/dynamic/ScalaSparkTest.java | 79 +++++++++++++++++++ 3 files changed, 119 insertions(+), 1 deletion(-) diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java index a151028..7e4549b 100644 --- a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoin.java @@ -55,7 +55,7 @@ public BroadcastJoin(BroadcastJoinConfig config) { public void configurePipeline(PipelineConfigurer pipelineConfigurer) { StageConfigurer stageConfigurer = pipelineConfigurer.getStageConfigurer(); Schema inputSchema = stageConfigurer.getInputSchema(); - if (inputSchema == null) { + if (inputSchema == null || !config.canCalculateOutputSchema()) { stageConfigurer.setOutputSchema(null); return; } diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java index a58e03a..33cbda2 100644 --- a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java @@ -48,12 +48,15 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter; + @Macro @Description("Schema of small dataset") private String datasetSchema; + @Macro @Description("Keys to join on") private String joinOn; + @Macro @Nullable @Description("Type of join") private String joinType; @@ -74,14 +77,17 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter2; + @Macro @Nullable @Description("Schema of small dataset") private String datasetSchema2; + @Macro @Nullable @Description("Keys to join on") private String joinOn2; + @Macro @Nullable @Description("Type of join") private String joinType2; @@ -100,14 +106,17 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter3; + @Macro @Nullable @Description("Schema of small dataset") private String datasetSchema3; + @Macro @Nullable @Description("Keys to join on") private String joinOn3; + @Macro @Nullable @Description("Type of join") private String joinType3; @@ -126,14 +135,17 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter4; + @Macro @Nullable @Description("Schema of small dataset") private String datasetSchema4; + @Macro @Nullable @Description("Keys to join on") private String joinOn4; + @Macro @Nullable @Description("Type of join") private String joinType4; @@ -152,14 +164,17 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter5; + @Macro @Nullable @Description("Schema of small dataset") private String datasetSchema5; + @Macro @Nullable @Description("Keys to join on") private String joinOn5; + @Macro @Nullable @Description("Type of join") private String joinType5; @@ -178,14 +193,17 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter6; + @Macro @Nullable @Description("Schema of small dataset") private String datasetSchema6; + @Macro @Nullable @Description("Keys to join on") private String joinOn6; + @Macro @Nullable @Description("Type of join") private String joinType6; @@ -204,14 +222,17 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter7; + @Macro @Nullable @Description("Schema of small dataset") private String datasetSchema7; + @Macro @Nullable @Description("Keys to join on") private String joinOn7; + @Macro @Nullable @Description("Type of join") private String joinType7; @@ -230,14 +251,17 @@ public class BroadcastJoinConfig extends PluginConfig { @Description("Delimiter used in the small dataset") private String delimiter8; + @Macro @Nullable @Description("Schema of small dataset") private String datasetSchema8; + @Macro @Nullable @Description("Keys to join on") private String joinOn8; + @Macro @Nullable @Description("Type of join") private String joinType8; @@ -250,6 +274,21 @@ public List getDatasetsToJoin() { return datasetJoinInfos; } + /** + * @return whether the output schema can be calculated without macro evaluation. + */ + public boolean canCalculateOutputSchema() { + for (int i = 0; i < numJoins; i++) { + String pathName = getPropertyName("path", i); + String schemaName = getPropertyName("datasetSchema", i); + String joinOnName = getPropertyName("joinOn", i); + if (containsMacro(pathName) || containsMacro(schemaName) || containsMacro(joinOnName)) { + return false; + } + } + return true; + } + private DatasetJoinInfo getJoinInfo(int datasetNum) { Map rawProperties = getProperties().getProperties(); String path = rawProperties.get(getPropertyName("path", datasetNum)); diff --git a/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java b/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java index 740e04b..bdc5ff8 100644 --- a/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java +++ b/src/test/java/io/cdap/plugin/spark/dynamic/ScalaSparkTest.java @@ -479,6 +479,85 @@ public void testBroadcastJoin() throws Exception { Assert.assertEquals("morgan", joined.get("lastname")); } + @Test + public void testBroadcastJoinMacros() throws Exception { + File smallDataset1 = TEMP_FOLDER.newFile("smallDataset1.csv"); + File smallDataset2 = TEMP_FOLDER.newFile("smallDataset2.csv"); + + try (FileWriter fileWriter = new FileWriter(smallDataset1)) { + fileWriter.write("0|;|alice\n"); + fileWriter.write("1|;|bob\n"); + } + try (FileWriter fileWriter = new FileWriter(smallDataset2)) { + fileWriter.write("alice,morgan\n"); + fileWriter.write("bob,vance\n"); + } + + Schema schema = Schema.recordOf("purchases", + Schema.Field.of("userid", Schema.of(Schema.Type.INT)), + Schema.Field.of("itemid", Schema.of(Schema.Type.LONG)), + Schema.Field.of("itemname", Schema.of(Schema.Type.STRING)), + Schema.Field.of("price", Schema.of(Schema.Type.DOUBLE))); + + Map properties = new HashMap<>(); + properties.put("numJoins", "2"); + properties.put("path", "${path1}"); + properties.put("datasetSchema", "${schema1}"); + properties.put("delimiter", "|;|"); + properties.put("joinOn", "${joinOn1}"); + properties.put("path2", "${path2}"); + properties.put("datasetSchema2", "${schema2}"); + properties.put("joinOn2", "${joinOn2}"); + String inputTableName = UUID.randomUUID().toString(); + String outputTableName = UUID.randomUUID().toString(); + ETLBatchConfig config = ETLBatchConfig.builder() + .addStage(new ETLStage("src", MockSource.getPlugin(inputTableName, schema))) + .addStage(new ETLStage("join", new ETLPlugin(BroadcastJoin.NAME, SparkCompute.PLUGIN_TYPE, properties))) + .addStage(new ETLStage("sink", MockSink.getPlugin(outputTableName))) + .addConnection("src", "join") + .addConnection("join", "sink") + .build(); + + ArtifactSummary artifactSummary = new ArtifactSummary(DATAPIPELINE_ARTIFACT_ID.getArtifact(), + DATAPIPELINE_ARTIFACT_ID.getVersion()); + AppRequest appRequest = new AppRequest<>(artifactSummary, config); + ApplicationId appId = NamespaceId.DEFAULT.app("testBroadcastJoin"); + ApplicationManager appManager = deployApplication(appId, appRequest); + + // setup input data + StructuredRecord record = StructuredRecord.builder(schema) + .set("userid", 0) + .set("itemid", 1000L) + .set("itemname", "donut") + .set("price", 1.0d) + .build(); + List inputRecords = Arrays.asList(record); + DataSetManager
inputTableManager = getDataset(inputTableName); + MockSource.writeInput(inputTableManager, inputRecords); + + Map runtimeArgs = new HashMap<>(); + runtimeArgs.put("path1", smallDataset1.getAbsolutePath()); + runtimeArgs.put("schema1", "userid int, username string"); + runtimeArgs.put("joinOn1", "userid"); + runtimeArgs.put("path2", smallDataset2.getAbsolutePath()); + runtimeArgs.put("schema2", "username string, lastname string"); + runtimeArgs.put("joinOn2", "username"); + + WorkflowManager workflowManager = appManager.getWorkflowManager(SmartWorkflow.NAME); + workflowManager.startAndWaitForRun(runtimeArgs, ProgramRunStatus.COMPLETED, 3, TimeUnit.MINUTES); + + DataSetManager
outputTableManager = getDataset(outputTableName); + List output = MockSink.readOutput(outputTableManager); + Assert.assertEquals(1, output.size()); + StructuredRecord joined = output.iterator().next(); + Assert.assertEquals(0, (int) joined.get("userid")); + Assert.assertEquals("donut", joined.get("itemname")); + Assert.assertEquals(1.0d, (double) joined.get("price"), 0.000001); + Assert.assertEquals(1000L, (long) joined.get("itemid")); + Assert.assertEquals("alice", joined.get("username")); + Assert.assertEquals("morgan", joined.get("lastname")); + } + private void testWordCountSink(String code, File outputFolder) throws Exception { Schema inputSchema = Schema.recordOf( "input", From 0a943777d302aeebf2fec914ff2e22ea119f17bb Mon Sep 17 00:00:00 2001 From: albertshau Date: Mon, 9 Mar 2020 09:52:19 -0700 Subject: [PATCH 4/4] set output schema when path is macro --- .../io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java index 33cbda2..f8e9ac7 100644 --- a/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java +++ b/src/main/java/io/cdap/plugin/spark/dynamic/BroadcastJoinConfig.java @@ -279,10 +279,9 @@ public List getDatasetsToJoin() { */ public boolean canCalculateOutputSchema() { for (int i = 0; i < numJoins; i++) { - String pathName = getPropertyName("path", i); String schemaName = getPropertyName("datasetSchema", i); String joinOnName = getPropertyName("joinOn", i); - if (containsMacro(pathName) || containsMacro(schemaName) || containsMacro(joinOnName)) { + if (containsMacro(schemaName) || containsMacro(joinOnName)) { return false; } } @@ -291,8 +290,9 @@ public boolean canCalculateOutputSchema() { private DatasetJoinInfo getJoinInfo(int datasetNum) { Map rawProperties = getProperties().getProperties(); - String path = rawProperties.get(getPropertyName("path", datasetNum)); - if (path == null) { + String pathNameStr = getPropertyName("path", datasetNum); + String path = rawProperties.get(pathNameStr); + if (!containsMacro(pathNameStr) && path == null) { throw new IllegalArgumentException("Path for Dataset " + datasetNum + " must be specified."); } pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy