diff --git a/README.md b/README.md index 02a21133e..435c025b2 100644 --- a/README.md +++ b/README.md @@ -6,7 +6,8 @@ **Coral** is a library for analyzing, processing, and rewriting views defined in the Hive Metastore, and sharing them across multiple execution engines. It performs SQL translations to enable views expressed in HiveQL (and potentially other languages) to be accessible in engines such as [Presto](https://prestosql.io/), -[Apache Spark](https://spark.apache.org/), and [Apache Pig](https://pig.apache.org/). +[Apache Spark](https://spark.apache.org/), [Apache Pig](https://pig.apache.org/), +or Java Streaming API with [Apache Beam](https://beam.apache.org/). Coral not only translates view definitions between different SQL/non-SQL dialects, but also rewrites expressions to produce semantically equivalent ones, taking into account the semantics of the target language or engine. For example, it automatically composes new built-in expressions that are equivalent to each built-in expression in the @@ -23,6 +24,8 @@ and implementing query rewrite algorithms for data governance and query optimiza - Coral-Pig: Converts view logical plan to Pig-latin. - Coral-Schema: Derives Avro schema of view using view logical plan and input Avro schemas of base tables. - Coral-Spark-Plan: Converts Spark plan strings to equivalent logical plan (in progress). +- Coral-Beam: Convert view logical plan to Beam Java API streaming code +- Coral-Beam-Runtime: runtime module to support Coral-Beam ## How to Build Clone the repository: @@ -40,3 +43,4 @@ Please see the [Contribution Agreement](CONTRIBUTING.md). ## Resources - [Coral & Transport UDFs: Building Blocks of a Postmodern Data Warehouse](https://www.slideshare.net/walaa_eldin_moustafa/coral-transport-udfs-building-blocks-of-a-postmodern-data-warehouse-229545076), Tech-talk, Facebook HQ, 2/28/2020. +- [Bridging Offline and Nearline Computations with Apache Calcite](https://engineering.linkedin.com/blog/2019/01/bridging-offline-and-nearline-computations-with-apache-calcite), LinkedIn Engineering Blogs 01/29/2019. diff --git a/build.gradle b/build.gradle index 22f413747..7a00e27be 100644 --- a/build.gradle +++ b/build.gradle @@ -26,6 +26,10 @@ allprojects { maven { url 'https://linkedin.bintray.com/maven/' } + maven { + url "https://packages.confluent.io/maven/" + content { includeGroup "io.confluent" } + } } } diff --git a/coral-beam-runtime/build.gradle b/coral-beam-runtime/build.gradle new file mode 100644 index 000000000..56c03d24c --- /dev/null +++ b/coral-beam-runtime/build.gradle @@ -0,0 +1,14 @@ +apply plugin: 'java' + +dependencies { + compile deps.'beam'.'java-core' + compile deps.'beam'.'java-io-kafka' + compile deps.'pig'.'pig' + compile deps.'hadoop'.'hadoop-common' + compile 'org.apache.avro:avro:1.7.7' + compile 'log4j:log4j:1.2.17' +} + +artifacts { + archives jar, javadocJar, sourcesJar +} \ No newline at end of file diff --git a/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamAPIUtil.java b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamAPIUtil.java new file mode 100644 index 000000000..4277fdd06 --- /dev/null +++ b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamAPIUtil.java @@ -0,0 +1,52 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import java.util.List; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.log4j.Logger; +import org.joda.time.Instant; + + +public class BeamAPIUtil { + private static final Logger LOG = Logger.getLogger(BeamAPIUtil.class.getName()); + + private BeamAPIUtil() { + } + + public static SerializableFunction, Instant> withTimeFunction(List timeField) { + return (SerializableFunction, Instant>) input -> { + final Instant now = Instant.now(); + if (timeField == null || timeField.isEmpty()) { + // No time field specified return current processing time + return now; + } + GenericRecord record = input.getValue(); + for (int i = 0; i < timeField.size(); i++) { + Object value = record.get(timeField.get(i)); + if (value == null) { + LOG.warn("Cannot get event time for input record, use current processing time instead. Time field: " + + timeField + ". Record: " + input.getValue()); + return now; + } + if (i < timeField.size() - 1) { + if (!(value instanceof GenericRecord)) { + throw new RuntimeException( + "Invalid schema for time field. Time field: " + timeField + ". Record schema: " + input.getValue().getSchema()); + } + record = (GenericRecord) value; + continue; + } + return new Instant(value); + } + + // Never reach here + return null; + }; + } +} diff --git a/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamArrayFlatten.java b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamArrayFlatten.java new file mode 100644 index 000000000..d3ff78ebb --- /dev/null +++ b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamArrayFlatten.java @@ -0,0 +1,153 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.values.KV; + + +public class BeamArrayFlatten { + private BeamArrayFlatten() { + } + + /** + * Flattens a Beam KV pair of String and Avro record on a given of array columns in the Avro record. These columns + * must have ARRAY schema type and the input values for those column must be either Java array or Collection. + * + * @param inputKV Input Beam KV pair of String and Avro record + * @param flattenCols Columns to be flattened + * @param outputSchema Schemas of ouput records + * @return Collection of output records. + */ + public static Collection> flatten(KV inputKV, + List flattenCols, Schema outputSchema) { + final List> results = new ArrayList<>(); + final String key = inputKV.getKey(); + final Collection resultRecords = flatten(inputKV.getValue(), flattenCols, outputSchema); + for (GenericRecord record : resultRecords) { + results.add(KV.of(key, record)); + } + return results; + } + + /** + * Flattens a record on a given of array columns. These columns must have ARRAY schema type + * and the input values for those column must be either Java array or Collection. + * + * @param inputRecord Input Avro records + * @param flattenCols Columns to be flattened + * @param outputSchema Schemas of ouput records + * @return Collection of output records. + */ + public static Collection flatten(GenericRecord inputRecord, List flattenCols, + Schema outputSchema) { + List results = new ArrayList<>(); + if (inputRecord == null) { + return results; + } + Schema inputSchema = inputRecord.getSchema(); + Map partialRecord = new HashMap<>(); + for (Schema.Field field : inputSchema.getFields()) { + partialRecord.put(field.name(), inputRecord.get(field.name())); + } + List> partialResults = new ArrayList<>(); + partialResults.add(partialRecord); + + for (String flattenCol : flattenCols) { + Schema flattenSchema = getFlattenElementSchema(inputSchema, flattenCol); + if (flattenSchema == null) { + throw new RuntimeException("Column " + flattenCol + " is not an array in records" + + " with schema: " + inputSchema); + } + Object obj = inputRecord.get(flattenCol); + List arrayValue = null; + if (obj != null) { + if (obj.getClass().isArray()) { + arrayValue = Arrays.asList((Object[]) obj); + } else if (obj instanceof Collection) { + if (obj instanceof List) { + arrayValue = (List) obj; + } else { + arrayValue = new ArrayList<>((Collection) obj); + } + } else { + throw new RuntimeException("Invalid avro array value. Value type: " + obj.getClass().getName()); + } + } + partialResults = extendFlattenRecords(partialResults, flattenCol, arrayValue, flattenSchema); + } + + for (Map outputRec : partialResults) { + GenericRecord record = new GenericData.Record(outputSchema); + for (Schema.Field field : outputSchema.getFields()) { + record.put(field.name(), outputRec.get(field.name())); + } + results.add(record); + } + return results; + } + + private static List> extendFlattenRecords(List> partialResults, + String flattenCol, List flattenValue, Schema flattenSchema) { + // Special optimization: no need to create new Map and List if array has only one value + if (flattenValue == null || flattenValue.size() <= 1) { + Object value = (flattenValue == null || flattenValue.isEmpty()) ? null : flattenValue.get(0); + for (Map partialRecord : partialResults) { + projectNestedFields(value, flattenCol, flattenSchema, partialRecord); + } + return partialResults; + } + + List> results = new ArrayList<>(); + for (Map partialRecord : partialResults) { + for (Object value : flattenValue) { + Map newRecord = new HashMap<>(partialRecord); + projectNestedFields(value, flattenCol, flattenSchema, newRecord); + results.add(newRecord); + } + } + return results; + } + + private static Schema getFlattenElementSchema(Schema recordSchema, String flattenCol) { + if (recordSchema.getField(flattenCol) == null) { + return null; + } + Schema colSchema = recordSchema.getField(flattenCol).schema(); + if (colSchema.getType() == Schema.Type.ARRAY) { + return colSchema.getElementType(); + } + if (colSchema.getType() == Schema.Type.UNION) { + for (Schema typeSchema : colSchema.getTypes()) { + if (typeSchema.getType() == Schema.Type.ARRAY) { + return typeSchema.getElementType(); + } + } + } + + return null; + } + + private static void projectNestedFields(Object nestedValue, String flattenCol, Schema flattenSchema, + Map partialRecord) { + if (flattenSchema.getType() == Schema.Type.RECORD) { + for (Schema.Field field : flattenSchema.getFields()) { + Object fieldValue = (nestedValue != null) ? ((GenericRecord) nestedValue).get(field.name()) : null; + partialRecord.put(field.name(), fieldValue); + } + } else { + partialRecord.put(flattenCol, nestedValue); + } + } +} diff --git a/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamExecUtil.java b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamExecUtil.java new file mode 100644 index 000000000..de1fed60d --- /dev/null +++ b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/BeamExecUtil.java @@ -0,0 +1,281 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import java.nio.ByteBuffer; +import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.Objects; +import java.util.TimeZone; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericFixed; +import org.apache.avro.generic.GenericRecord; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.builtin.Nondeterministic; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.util.avro.AvroStorageDataConversionUtilities; + + +public class BeamExecUtil { + public static final String RECORD_NAME_PREFIXED = "record0"; + public static final String DYNAMIC_RECORD_NAME_PREFIXED = "dynamic_record"; + + private BeamExecUtil() { + } + + private static GenericRecord dynamicTuple2Avro(final Tuple t, final Schema s) throws ExecException { + final GenericData.Record record = new GenericData.Record(s); + for (Schema.Field f : s.getFields()) { + final Object o = t.get(f.pos()); + record.put(f.pos(), o); + } + return record; + } + + /** + * Converts an object, which can be already ByteBuffer or GenericFixed, + * into a ByteBuffer. Returns null if object is null. + */ + public static ByteBuffer toByteBuffer(Object object) { + if (object == null) { + return null; + } + + if (object instanceof ByteBuffer) { + return (ByteBuffer) object; + } + if (object instanceof GenericFixed) { + return ByteBuffer.wrap(((GenericFixed) object).bytes()); + } + throw new RuntimeException("Cannot convert " + object.getClass().getName() + " object into ByteBuffer"); + } + + /** + * Converts an object, which can be already an Avro GenericRecord or a Pig Tuple, + * into an Avro GenericRecord. Returns null if object is null. + */ + public static GenericRecord toAvroRecord(Object obj, Schema schema) { + if (obj == null) { + return null; + } + + if (obj instanceof GenericRecord) { + return (GenericRecord) obj; + } + + if (obj instanceof Tuple) { + final Tuple tuple = (Tuple) obj; + try { + if (isDynamicRecord(schema)) { + return dynamicTuple2Avro(tuple, schema); + } else { + return AvroStorageDataConversionUtilities.packIntoAvro(tuple, schema); + } + } catch (Exception e) { + throw new RuntimeException("Cannot convert " + obj + " into GenericRecord", e); + } + } + throw new RuntimeException("Cannot convert " + obj.getClass().getName() + " object into GenericRecord"); + } + + /** + * Converts an object, which can be already an Avro GenericArray or a Pig DataBag, + * into an Avro GenericArray. Returns null if object is null. + */ + public static GenericArray toAvroArray(Object obj, Schema schema) { + if (obj == null) { + return null; + } + + if (obj instanceof GenericArray) { + return (GenericArray) obj; + } + + if (obj instanceof DataBag) { + final DataBag bag = (DataBag) obj; + final GenericData.Array array = new GenericData.Array<>(new Long(bag.size()).intValue(), schema); + for (Tuple tuple : bag) { + try { + if (schema.getElementType() != null && schema.getElementType().getType() == Schema.Type.RECORD) { + array.add(toAvroRecord(tuple, schema.getElementType())); + } else if (tuple.size() == 1) { + array.add(tuple.get(0)); + } else { + throw new RuntimeException("Can't pack " + tuple + " into schema " + schema); + } + } catch (ExecException e) { + throw new RuntimeException("Can't pack " + tuple + " into schema " + schema, e); + } + } + return array; + } + + throw new RuntimeException("Cannot convert " + obj.getClass().getName() + " object into GenericArray"); + } + + private static boolean isDynamicRecord(Schema schema) { + return schema.getName() != null && schema.getName().startsWith(DYNAMIC_RECORD_NAME_PREFIXED); + } + + /** + * Wrapper for Math.random to indicate that it's nondeterministic + * @return + */ + @Nondeterministic + public static double rand() { + return Math.random(); + } + + public static String buildStringKey(String... stringList) { + StringBuilder builder = new StringBuilder(); + for (String item : stringList) { + if (item != null) { + builder.append(item); + } else { + builder.append("null"); + } + builder.append("_"); + } + return builder.toString(); + } + + public static String buildStringKey(GenericRecord record, String... keyColNames) { + if (keyColNames == null) { + return "null"; + } + String[] keys = new String[keyColNames.length]; + for (int i = 0; i < keyColNames.length; i++) { + keys[i] = Objects.toString(record.get(keyColNames[i])); + } + return buildStringKey(keys); + } + + public static String buildDistinctStringKeyFromRecord(GenericRecord avroRecord, String timestampCol) { + return buildDistinctStringKeyFromRecord("", avroRecord, timestampCol); + } + + public static String buildDistinctStringKeyFromRecord(String prefix, GenericRecord avroRecord, String timestampCol) { + final StringBuilder builder = new StringBuilder(); + builder.append(prefix); + for (Schema.Field field : avroRecord.getSchema().getFields()) { + // Ignore timestamp column + if (field.name().equalsIgnoreCase(timestampCol)) { + continue; + } + builder.append(Objects.toString(avroRecord.get(field.name()), "null")); + builder.append("_"); + } + return builder.toString(); + } + + public static String buildStringKeyFromRecord(GenericRecord avroRecord) { + final StringBuilder builder = new StringBuilder(); + for (Schema.Field field : avroRecord.getSchema().getFields()) { + builder.append(Objects.toString(avroRecord.get(field.name()), "null")); + builder.append("_"); + } + return builder.toString(); + } + + public static String getOpId(String opName, int codeVersion) { + return opName + "_v" + codeVersion; + } + + /** + * Rounds up time to a given range. Example roundTime = 15, timeUnit = MINUTE, epochTime will be rounded + * up to every 15 minutes, like 10:00, 10:15, 10:30... + * + * @param epochTime Original epoch time + * @param roundTime Amount of time to round up (see above example) + * @param timeUnit Time unit in {{@link Calendar}}, from HOUR_OF_DAY to MILLISECOND + * @return Time in epoch after rounding up + */ + public static long roundUpTime(long epochTime, int roundTime, int timeUnit) { + return roundUpTimeInCalendar(epochTime, roundTime, timeUnit).getTimeInMillis(); + } + + private static Calendar roundUpTimeInCalendar(long epochTime, int roundTime, int timeUnit) { + if (epochTime <= 0) { + throw new IllegalArgumentException("Invalid time to convert: " + epochTime); + } + + if (roundTime <= 0) { + throw new IllegalArgumentException("Invalid round up time: " + roundTime); + } + + if (timeUnit > Calendar.MILLISECOND || (timeUnit < Calendar.HOUR_OF_DAY && timeUnit != Calendar.DATE)) { + throw new IllegalArgumentException("Invalid time unit: " + timeUnit); + } + + final Calendar cal = getCalendarFromEpoch(epochTime); + cal.set(timeUnit, cal.get(timeUnit) - (cal.get(timeUnit) % roundTime)); + for (int unit = Calendar.MILLISECOND; unit > Math.max(timeUnit, Calendar.HOUR); --unit) { + cal.set(unit, 0); + } + return cal; + } + + /** + * Checks if end time is expired from the begin time after a duration. + * + * Example: + * - 2019/03/20 23:59, 2019/03/21 00:00, DATE, 1 - true + * - 2019/03/20 23:59, 2019/03/21 00:00, MINUTE, 2 - false + * - 2019/03/21 00:00, 2019/03/21 23:59, DAY, 1 - false + * - 2019/03/21 00:59, 2019/03/21 01:00, HOUR_OF_DAY, 2 - false + * + * @param begin begin time + * @param end end time + * @param timeGranularity Time granularity unit (in {{@link Calendar}}, from HOUR_OF_DAY to MILLISECOND) + * for rounding up and comparison + * @param duration Duration (within time granularity) for expiration + * @return true iff end time is expired + */ + public static boolean isExpired(long begin, long end, int timeGranularity, int duration) { + final Calendar beginCal = roundUpTimeInCalendar(begin, 1, timeGranularity); + final Calendar endCal = roundUpTimeInCalendar(end, 1, timeGranularity); + beginCal.add(timeGranularity, duration - 1); + return beginCal.before(endCal); + } + + private static Calendar getCalendarFromEpoch(long epochTime) { + final Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC")); + calendar.setTimeInMillis(epochTime); + return calendar; + } + + /** + * Converts epoch time to yyyyMMdd format in Pacific timezone + * + * @param epochTime epoch time + */ + public static Long toDateFormat(long epochTime) { + return toDateFormat(epochTime, "America/Los_Angeles"); + } + + /** + * Converts epoch time to yyyyMMdd format in a given timezone + * + * @param epochTime epoch time + * @param timeZone time zone + */ + public static Long toDateFormat(long epochTime, String timeZone) { + final SimpleDateFormat dateParser = new SimpleDateFormat("yyyyMMdd"); + dateParser.setTimeZone(TimeZone.getTimeZone(timeZone)); + if (epochTime < 0) { + return null; + } + + if (epochTime < 10000000000L) { + return Long.parseLong(dateParser.format(epochTime * 1000)); + } + + return Long.parseLong(dateParser.format(epochTime)); + } +} diff --git a/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/DedupFn.java b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/DedupFn.java new file mode 100644 index 000000000..1bd0f7477 --- /dev/null +++ b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/DedupFn.java @@ -0,0 +1,66 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import java.time.Instant; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.joda.time.Duration; + + +public class DedupFn extends DoFn, KV> { + private static final String STATE_ID = "recordDedup"; + + private final Duration dedupWindow; + + @StateId(STATE_ID) + private final StateSpec> lassModifiedTime = StateSpecs.value(VarLongCoder.of()); + + public DedupFn(Duration dedupWindow) { + this.dedupWindow = dedupWindow; + } + + @ProcessElement + public void processElement( + ProcessContext context, @StateId(STATE_ID) ValueState lassModifiedTime) { + final Long lastTouched = lassModifiedTime.read(); + final long eventTime = context.timestamp().getMillis(); + final long currentTime = eventTime > 0 ? eventTime : Instant.now().toEpochMilli(); + if (lastTouched == null) { + // Output record if we have not seen it + context.output(context.element()); + lassModifiedTime.write(currentTime); + } else if (isExpired(lastTouched, currentTime, dedupWindow)) { + // If out of date, emit and remove it + context.output(context.element()); + lassModifiedTime.clear(); + } else { + // Else (seen record and it is still valid) just update last accessed time + lassModifiedTime.write(currentTime); + } + } + + /** + * Checks if end time is expired from the begin time, given a window. + * + *

Example: - 2019/03/20 23:59, 2019/03/21 00:00, 1 DAY -> true - 2019/03/20 23:59, 2019/03/21 + * 00:00, 2 MINUTES -> false - 2019/03/21 00:00, 2019/03/21 23:59, 1 DAY -> false - 2019/03/21 + * 00:59, 2019/03/21 01:00, 2 HOURS -> false + * + * @param begin begin time + * @param end end time + * @param window Duration for the window to compare + * @return true iff begin time is in the earlier window than end time + */ + private static boolean isExpired(long begin, long end, Duration window) { + long windowMillis = window.getMillis(); + return begin / windowMillis < end / windowMillis; + } +} diff --git a/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/KafkaIOGenericRecord.java b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/KafkaIOGenericRecord.java new file mode 100644 index 000000000..995e3552e --- /dev/null +++ b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/KafkaIOGenericRecord.java @@ -0,0 +1,23 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.io.kafka.KafkaIO; + + +public class KafkaIOGenericRecord { + public static KafkaIO.Read read() { + return KafkaIO.read(); + } + + public static KafkaIO.Write write() { + return KafkaIO.write(); + } + + private KafkaIOGenericRecord() { + } +} diff --git a/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/MessageDedup.java b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/MessageDedup.java new file mode 100644 index 000000000..fdf08524a --- /dev/null +++ b/coral-beam-runtime/src/main/java/com/linkedin/beam/excution/MessageDedup.java @@ -0,0 +1,50 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Reshuffle; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; + + +/** This class performs message deduplication on keys. */ +public class MessageDedup extends PTransform>, PCollection>> { + // Window for deduping event. Events within a same window will be deduped. + private final Duration dedupWindow; + + private MessageDedup(Duration dedupWindow) { + this.dedupWindow = dedupWindow; + } + + /** + * Returns a transform that will dedup a stream by key in fixed windows. The transformation acts + * as a filter to skip any messages with keys that have been seen in the same window and output + * them otherwise. Messages with new key will be save into the internal state. + * + *

Usage: + * + *

{@code
+   * input.apply(MessageDedup.within(Duration.standardHours(2));
+   * }
+ * + * This will dedup all messages with the same 2 hour window, i.e. from 8:00-10:00, 10:00-12:00. + * + * @param dedupWindow Duration for deduping window + * @param Key type + * @param Value type + */ + public static MessageDedup within(Duration dedupWindow) { + return new MessageDedup<>(dedupWindow); + } + + @Override + public PCollection> expand(PCollection> input) { + return input.apply(Reshuffle.of()).apply(ParDo.of((new DedupFn<>(dedupWindow)))); + } +} diff --git a/coral-beam-runtime/src/test/java/com/linkedin/beam/excution/ArrayFlattenTest.java b/coral-beam-runtime/src/test/java/com/linkedin/beam/excution/ArrayFlattenTest.java new file mode 100644 index 000000000..d911e181e --- /dev/null +++ b/coral-beam-runtime/src/test/java/com/linkedin/beam/excution/ArrayFlattenTest.java @@ -0,0 +1,116 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import com.google.common.collect.Sets; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.testng.Assert; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + + +public class ArrayFlattenTest { + private static final Schema INPUT_SCHEMA = new Schema.Parser().parse( + "{\"type\":\"record\"," + + "\"name\":\"record0\"," + + "\"namespace\":\"rel_avro\"," + + "\"fields\":[" + + "{\"name\":\"col0\",\"type\":[\"string\",\"null\"]}," + + "{\"name\":\"col1\",\"type\":{\"type\":\"array\",\"items\":\"int\"}}," + + "{\"name\":\"col2\"," + + "\"type\":{\"type\":\"array\"," + + "\"items\":{\"type\":\"record\",\"name\":\"nestedRec\"," + + "\"fields\":[" + + "{\"name\":\"field1\",\"type\":[\"string\",\"null\"]}," + + "{\"name\":\"field2\",\"type\":[\"int\",\"null\"]}" + + "]" + + "}" + + "}" + + "}" + + "]}"); + + private static final Schema NESTED_SCHEMA = new Schema.Parser().parse( + "{\"type\":\"record\",\"name\":\"nestedRec\"," + + "\"fields\":[" + + "{\"name\":\"field1\",\"type\":[\"string\",\"null\"]}," + + "{\"name\":\"field2\",\"type\":[\"int\",\"null\"]}" + + "] }"); + + private static final Schema OUTPUT_SCHEMA = new Schema.Parser().parse( + "{\"type\":\"record\"," + + "\"name\":\"record1\"," + + "\"namespace\":\"rel_avro\"," + + "\"fields\":[" + + "{\"name\":\"col0\",\"type\":[\"string\",\"null\"]}," + + "{\"name\":\"col1\",\"type\":[\"int\",\"null\"]}," + + "{\"name\":\"field1\",\"type\":[\"string\",\"null\"]}," + + "{\"name\":\"field2\",\"type\":[\"int\",\"null\"]}" + + "]}"); + + private static final List FLATTEN_COLS = Arrays.asList("col1", "col2"); + + private static final String RESULT1 = "[" + + "{\"col0\": \"rec1\", \"col1\": 1, \"field1\": null, \"field2\": null}, " + + "{\"col0\": \"rec1\", \"col1\": 2, \"field1\": null, \"field2\": null}, " + + "{\"col0\": \"rec1\", \"col1\": 3, \"field1\": null, \"field2\": null}]"; + private static final String RESULT2 = "[" + + "{\"col0\": null, \"col1\": 4, \"field1\": \"nested1\", \"field2\": 1}, " + + "{\"col0\": null, \"col1\": 4, \"field1\": \"nested2\", \"field2\": 2}]"; + private static final String RESULT3 = "[" + + "{\"col0\": \"rec3\", \"col1\": 6, \"field1\": \"nested1\", \"field2\": 1}, " + + "{\"col0\": \"rec3\", \"col1\": 6, \"field1\": \"nested2\", \"field2\": 2}, " + + "{\"col0\": \"rec3\", \"col1\": 7, \"field1\": \"nested1\", \"field2\": 1}, " + + "{\"col0\": \"rec3\", \"col1\": 7, \"field1\": \"nested2\", \"field2\": 2}, " + + "{\"col0\": \"rec3\", \"col1\": 8, \"field1\": \"nested1\", \"field2\": 1}, " + + "{\"col0\": \"rec3\", \"col1\": 8, \"field1\": \"nested2\", \"field2\": 2}]"; + + private GenericRecord record1; + private GenericRecord record2; + private GenericRecord record3; + + @BeforeMethod + public void setUp() { + record1 = new GenericData.Record(INPUT_SCHEMA); + record1.put("col0", "rec1"); + record1.put("col1", new Integer[] { 1, 2, 3}); + + record2 = new GenericData.Record(INPUT_SCHEMA); + record2.put("col1", Sets.newHashSet(4)); + GenericRecord nestedRec1 = new GenericData.Record(NESTED_SCHEMA); + nestedRec1.put("field1", "nested1"); + nestedRec1.put("field2", 1); + GenericRecord nestedRec2 = new GenericData.Record(NESTED_SCHEMA); + nestedRec2.put("field1", "nested2"); + nestedRec2.put("field2", 2); + List nestedValues = Arrays.asList(nestedRec1, nestedRec2); + record2.put("col2", nestedValues); + + record3 = new GenericData.Record(INPUT_SCHEMA); + record3.put("col0", "rec3"); + record3.put("col1", new Integer[] { 6, 7, 8}); + record3.put("col2", nestedValues); + + + } + + private void testFlattenHelper(GenericRecord inputRec, int expectedSize, String expectedResult) { + Collection results = BeamArrayFlatten.flatten(inputRec, FLATTEN_COLS, OUTPUT_SCHEMA); + Assert.assertEquals(results.size(), expectedSize); + Assert.assertEquals(results.toString(), expectedResult); + } + + @Test + public void testFlatten() { + testFlattenHelper(record1, 3, RESULT1); + testFlattenHelper(record2, 2, RESULT2); + testFlattenHelper(record3, 6, RESULT3); + } +} diff --git a/coral-beam-runtime/src/test/java/com/linkedin/beam/excution/TestExecUtil.java b/coral-beam-runtime/src/test/java/com/linkedin/beam/excution/TestExecUtil.java new file mode 100644 index 000000000..15f435068 --- /dev/null +++ b/coral-beam-runtime/src/test/java/com/linkedin/beam/excution/TestExecUtil.java @@ -0,0 +1,175 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.excution; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; +import java.util.TimeZone; +import java.util.stream.Collectors; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; + +import org.codehaus.jackson.node.NullNode; +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class TestExecUtil { + + private Calendar getCalendar(String timezone) { + return Calendar.getInstance(TimeZone.getTimeZone(timezone)); + } + private Calendar getCalendar() { + return getCalendar("UTC"); + } + + + private static class Pair { + K key; + V value; + + public Pair(K key, V value) { + this.key = key; + this.value = value; + } + } + + static Schema createSchema(List> fieldTypes) { + List fields = + fieldTypes.stream() + .map(ntp -> new Schema.Field(ntp.key, Schema.create(ntp.value), "", + NullNode.getInstance())) + .collect(Collectors.toList()); + return Schema.createRecord(fields); + } + + @Test + public void testTimeRoundUp() { + Calendar cal = getCalendar(); + cal.set(2018, 7, 25, 14, 48, 30); // 2018-08-25 14:40:30 + long inputTime = cal.getTimeInMillis(); + testTimeRoundUpHelper(inputTime, Calendar.MINUTE, 15, 45); + testTimeRoundUpHelper(inputTime, Calendar.MINUTE, 10, 40); + testTimeRoundUpHelper(inputTime, Calendar.MINUTE, 8, 48); + testTimeRoundUpHelper(inputTime, Calendar.HOUR_OF_DAY, 3, 12); + + Assert.assertEquals(BeamExecUtil.roundUpTime(inputTime, 1, Calendar.DATE), 1535155200000L /* 2018-08-25 00:00:00 */); + Assert.assertEquals(BeamExecUtil.roundUpTime(inputTime, 2, Calendar.DATE), 1535068800000L /* 2018-08-24 00:00:00 */); + } + + private void testTimeRoundUpHelper(long inputTime, int timeUnit, int roundTime, int expected) { + Calendar out = getCalendar(); + out.setTimeInMillis(BeamExecUtil.roundUpTime(inputTime, roundTime, timeUnit)); + Assert.assertEquals(out.get(timeUnit), expected); + for (int unit = Calendar.MILLISECOND; unit > timeUnit; --unit) { + Assert.assertEquals(out.get(unit), 0); + } + } + + @Test + public void testExpiredTime() { + Calendar cal1 = getCalendar(); + Calendar cal2 = getCalendar(); + + cal1.set(2019, 3, 20, 23, 59, 59); + cal2.set(2019, 3, 21, 0, 0, 0); + Assert.assertTrue(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.DATE, 1)); + Assert.assertFalse(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.DATE, 2)); + cal2.set(Calendar.DATE, 20); + Assert.assertFalse(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.HOUR_OF_DAY, 1)); + + cal1.set(2019, 3, 20, 0, 59, 59); + cal2.set(2019, 3, 20, 2, 0, 0); + Assert.assertTrue(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.HOUR_OF_DAY, 2)); + Assert.assertFalse(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.HOUR_OF_DAY, 3)); + cal2.set(Calendar.HOUR_OF_DAY, 0); + Assert.assertFalse(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.HOUR_OF_DAY, 1)); + + cal1.set(2019, 3, 21, 0, 0, 0); + cal2.set(2019, 3, 21, 0, 0, 0); + cal1.set(Calendar.MILLISECOND, 0); + + cal2.set(Calendar.MILLISECOND, 99); + Assert.assertFalse(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.MILLISECOND, 100)); + cal2.set(Calendar.MILLISECOND, 100); + Assert.assertTrue(BeamExecUtil.isExpired(cal1.getTimeInMillis(), cal2.getTimeInMillis(), Calendar.MILLISECOND, 100)); + } + + @Test + public void testToDateFormat() { + // Test pacific time + Calendar cal1 = getCalendar("America/Los_Angeles"); + cal1.set(2019, 3, 21, 0, 0, 0); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal1.getTimeInMillis()), 20190421); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal1.getTimeInMillis(), "UTC"), 20190421); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal1.getTimeInMillis() / 1000), 20190421); + cal1.set(2019, 3, 21, 23, 59, 59); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal1.getTimeInMillis()), 20190421); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal1.getTimeInMillis(), "UTC"), 20190422); + + // Test UTC time + Calendar cal2 = getCalendar(); + cal2.set(2019, 3, 21, 0, 0, 0); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal2.getTimeInMillis()), 20190420); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal2.getTimeInMillis(), "UTC"), 20190421); + cal2.set(2019, 3, 21, 23, 59, 59); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal2.getTimeInMillis()), 20190421); + Assert.assertEquals((long) BeamExecUtil.toDateFormat(cal2.getTimeInMillis(), "UTC"), 20190421); + } + + private GenericRecord createRecord(String stringCol, Integer intCol, Long longCol, Double doubleCol, Boolean boolCol, Long timestamp) { + List> fields = new ArrayList<>(); + fields.add(new Pair<>("stringCol", Schema.Type.STRING)); + fields.add(new Pair<>("intCol", Schema.Type.INT)); + fields.add(new Pair<>("longCol", Schema.Type.LONG)); + fields.add(new Pair<>("doubleCol", Schema.Type.DOUBLE)); + fields.add(new Pair<>("boolCol", Schema.Type.BOOLEAN)); + fields.add(new Pair<>("timestamp", Schema.Type.LONG)); + GenericRecord record = new GenericData.Record(createSchema(fields)); + record.put("stringCol", stringCol); + record.put("intCol", intCol); + record.put("longCol", longCol); + record.put("doubleCol", doubleCol); + record.put("boolCol", boolCol); + record.put("timestamp", timestamp); + return record; + } + + private void testBuildKeyHelper(String stringCol, Integer intCol, Long longCol, Double doubleCol, Boolean boolCol, Long timestamp, String expected) { + GenericRecord record = createRecord(stringCol, intCol, longCol, doubleCol, boolCol, timestamp); + Assert.assertEquals(BeamExecUtil.buildStringKeyFromRecord(record), expected); + } + + @Test + public void testBuildKeyFromRecord() { + testBuildKeyHelper("test", 1, 2L, 3.01, true, 123L, "test_1_2_3.01_true_123_"); + testBuildKeyHelper("test", null, 2L, 3.01, false, 123L, "test_null_2_3.01_false_123_"); + testBuildKeyHelper("test", null, null, null, null, null, "test_null_null_null_null_null_"); + } + + private void testBuildDistinctKeyHelper(String stringCol, Integer intCol, Long longCol, Double doubleCol, + Boolean boolCol, Long timestamp, String expected) { + testBuildDistinctKeyHelper("", stringCol, intCol, longCol, doubleCol, boolCol, timestamp, expected); + } + + private void testBuildDistinctKeyHelper(String prefix, String stringCol, Integer intCol, Long longCol, + Double doubleCol, Boolean boolCol, Long timestamp, String expected) { + GenericRecord record = createRecord(stringCol, intCol, longCol, doubleCol, boolCol, timestamp); + Assert.assertEquals(BeamExecUtil.buildDistinctStringKeyFromRecord(prefix, record, "timestamp"), expected); + } + + @Test + public void testBuildDistinctKeyFromRecord() { + testBuildDistinctKeyHelper("test", 1, 2L, 3.01, true, 123L, "test_1_2_3.01_true_"); + testBuildDistinctKeyHelper("prefix_", "test", 1, 2L, 3.01, true, 123L, "prefix_test_1_2_3.01_true_"); + testBuildDistinctKeyHelper("test", null, 2L, 3.01, false, Instant.now().toEpochMilli(), "test_null_2_3.01_false_"); + testBuildDistinctKeyHelper("test", null, null, null, null, null, "test_null_null_null_null_"); + } +} diff --git a/coral-beam/build.gradle b/coral-beam/build.gradle new file mode 100644 index 000000000..3c27f1b01 --- /dev/null +++ b/coral-beam/build.gradle @@ -0,0 +1,12 @@ +apply plugin: 'java' + +dependencies { + compile project(path: ':coral-beam-runtime') + compile('com.linkedin.calcite:calcite-piglet:1.21.0.140') + compile('com.google.googlejavaformat:google-java-format:1.4') + testCompile deps.'hadoop'.'hadoop-mapreduce-client-core' +} + +artifacts { + archives jar, javadocJar, sourcesJar +} \ No newline at end of file diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/AggCallImplementation.java b/coral-beam/src/main/java/com/linkedin/beam/operators/AggCallImplementation.java new file mode 100644 index 000000000..abf2219ee --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/AggCallImplementation.java @@ -0,0 +1,250 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.linkedin.beam.utils.Methods; +import com.linkedin.beam.utils.RexBeamUtils; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.avro.generic.GenericData; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.rel.core.AggregateCall; + + +/** + * Implementation Java code for aggregate calls. + * + * To implement a new aggregate function, we need to add a new class extending from AggCallImplementation, + * implement these three methods: + * 1. init() to provide initial value for the aggregate variable + * 2. aggregate() to provide a list of statements to aggregate current input message into the aggregate variable + * 3. result() to provide the final result for the aggregation, + * and update the switch case statements in getAggCallImpl() to include the new aggregate function. + * + * We support two kinds of aggregate functions: + * 1. Builtin aggregate functions: we will generate code to do the aggregation. This is for simple aggregate + * functions like COUNT, MIN, MAX, SUM... + * 2. Custom aggregate function: the aggregate code is written in other libraries and is hooked up into the + * auto-generated code by appropriate method call. This is for complex aggregate functions like COUNT DISTINCT, + * QUANTILE,... + * + * + * See sample of a BuiltinAggCall in SumImplementation class + */ +public abstract class AggCallImplementation { + // Calcite aggregate call + AggregateCall aggCall; + // Variable expression for the aggregate + ParameterExpression aggVar; + // Map from input column index to a pair of column variable expression and the column value expression + Map> inputColMap; + + private AggCallImplementation(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + this.aggCall = aggCall; + this.aggVar = aggVar; + this.inputColMap = inputColMap; + } + + /** + * Gets aggregate call implementation for a given Calcite aggregate call. + */ + static AggCallImplementation getAggCallImpl(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + switch (aggCall.getAggregation().kind) { + case COUNT: + return new CountImplementation(aggCall, aggVar, inputColMap); + case SUM: + return new SumImplementation(aggCall, aggVar, inputColMap); + case MAX: + return new MaxImplementation(aggCall, aggVar, inputColMap); + case MIN: + return new MinImplementation(aggCall, aggVar, inputColMap); + case COLLECT: + return new CollectImplementation(aggCall, aggVar, inputColMap); + default: + throw new UnsupportedOperationException("Aggregate function " + aggCall.getAggregation().kind + + " not yet supported"); + } + } + + /** + * Returns an expression to initialize the aggregate variable + */ + abstract Expression init(); + + /** + * Returns statements for doing aggregate on each input record + */ + abstract List aggregate(); + + /** + * Returns an expression for the final result of the aggregation after consuming all input messages + */ + abstract Expression result(); + + // Adds null check for a statement. Code example: + // if (var != null) { + // ifBody + // } + private static Statement checkNotNull(Expression var, Statement ifBody) { + return Expressions.ifThen(Expressions.notEqual(var, Expressions.constant(null)), ifBody); + } + + private static abstract class BuiltinAggCall extends AggCallImplementation { + + private BuiltinAggCall(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + super(aggCall, aggVar, inputColMap); + } + + @Override + public Expression result() { + // The aggregate result is just the aggregate variable + return aggVar; + } + } + + private static class CollectImplementation extends BuiltinAggCall { + + public CollectImplementation(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + super(aggCall, aggVar, inputColMap); + } + + @Override + public Expression init() { + final ParameterExpression schema = RexBeamUtils.getSchemaParam(RexBeamUtils.getSchemaName(aggVar.name)); + return Expressions.new_(GenericData.Array.class, schema, Expressions.new_(ArrayList.class)); + } + + @Override + public List aggregate() { + final List results = new ArrayList<>(); + final Expression inputCol = inputColMap.get(aggCall.getArgList().get(0)).get(0); + final Statement addStatement = Expressions.statement( + Expressions.call(aggVar, Methods.AVRO_ARRAY_ADD, inputCol)); + results.add(addStatement); + return results; + } + } + + private static class SumImplementation extends BuiltinAggCall { + + public SumImplementation(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + super(aggCall, aggVar, inputColMap); + } + + @Override + public Expression init() { + return Expressions.constant(0); + } + + @Override + public List aggregate() { + final List results = new ArrayList<>(); + final Expression inputCol = inputColMap.get(aggCall.getArgList().get(0)).get(0); + final Statement addStatement = Expressions.statement( + Expressions.assign(aggVar, Expressions.add(aggVar, inputCol))); + results.add(checkNotNull(inputCol, addStatement)); + return results; + } + } + + private static class CountImplementation extends BuiltinAggCall { + + public CountImplementation(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + super(aggCall, aggVar, inputColMap); + } + + @Override + public Expression init() { + return Expressions.constant(0); + } + + @Override + public List aggregate() { + final List results = new ArrayList<>(); + final List argList = aggCall.getArgList(); + final Statement incStatement = Expressions.statement(Expressions.assign(aggVar, + Expressions.add(aggVar, Expressions.constant(1)))); + if (!argList.isEmpty()) { + // Example code: + // if (project2longCol != null) { + // aggregate3_f8 = aggregate3_f8 + 1; + // } + final Expression inputCol = inputColMap.get(aggCall.getArgList().get(0)).get(0); + results.add(checkNotNull(inputCol, incStatement)); + } else { + // COUNT(*), always inc + results.add(incStatement); + } + return results; + } + } + + private static abstract class MinMaxImplementation extends BuiltinAggCall { + + public MinMaxImplementation(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + super(aggCall, aggVar, inputColMap); + } + + @Override + public Expression init() { + // Init with the input column value of the first input record. Code example: + // long aggregate3_f2 = (Long) inputRecord.get("longCol"); + return inputColMap.get(aggCall.getArgList().get(0)).get(1); + } + + @Override + public List aggregate() { + final List results = new ArrayList<>(); + final Expression inputCol = inputColMap.get(aggCall.getArgList().get(0)).get(0); + final Expression comparisonExpr = getComparisonExpr(inputCol); + final Expression binaryCondExpr = Expressions.condition(comparisonExpr, aggVar, inputCol); + results.add(checkNotNull(inputCol, Expressions.statement(Expressions.assign(aggVar, binaryCondExpr)))); + return results; + } + + abstract Expression getComparisonExpr(Expression inputCol); + } + + private static class MaxImplementation extends MinMaxImplementation { + + public MaxImplementation(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + super(aggCall, aggVar, inputColMap); + } + + @Override + Expression getComparisonExpr(Expression inputCol) { + return inputCol.type != String.class ? Expressions.greaterThan(aggVar, inputCol) + : Expressions.greaterThan(RexBeamUtils.stringCompareExpr(aggVar, inputCol), Expressions.constant(0)); + } + + } + + private static class MinImplementation extends MinMaxImplementation { + public MinImplementation(AggregateCall aggCall, ParameterExpression aggVar, + Map> inputColMap) { + super(aggCall, aggVar, inputColMap); + } + + @Override + Expression getComparisonExpr(Expression inputCol) { + return inputCol.type != String.class ? Expressions.lessThan(aggVar, inputCol) + : Expressions.lessThan(RexBeamUtils.stringCompareExpr(aggVar, inputCol), Expressions.constant(0)); + } + + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamAggregate.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamAggregate.java new file mode 100644 index 000000000..e2b4faab4 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamAggregate.java @@ -0,0 +1,334 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.linkedin.beam.excution.BeamExecUtil; +import com.linkedin.beam.excution.MessageDedup; +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.planner.BeamPlanner; +import com.linkedin.beam.planner.CalciteBeamConfig; +import com.linkedin.beam.utils.MethodNames; +import com.linkedin.beam.utils.Methods; +import com.linkedin.beam.utils.RelDataTypeToAvro; +import com.linkedin.beam.utils.RexBeamUtils; +import com.linkedin.beam.utils.RexToBeamConverter; +import com.linkedin.beam.utils.AvroJavaTypeFactory; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.util.ImmutableBitSet; + +import static com.linkedin.beam.utils.MethodNames.*; + + +public class BeamAggregate extends Aggregate implements BeamNode { + private int beamNodeId; + private final long aggregateWindowMinutes; + + private BeamAggregate(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, + ImmutableBitSet groupSet, List groupSets, List aggCalls, + long aggregateWindowMinutes) { + super(cluster, traitSet, input, groupSet, groupSets, aggCalls); + this.aggregateWindowMinutes = aggregateWindowMinutes; + } + + public static BeamAggregate create(RelNode input, ImmutableBitSet groupSet, + List groupSets, List aggCalls, long aggregateWindowMinutes) { + final RelOptCluster cluster = input.getCluster(); + return new BeamAggregate(cluster, cluster.traitSetOf(BeamConvention.INSTANCE), input, groupSet, + groupSets, aggCalls, aggregateWindowMinutes); + } + @Override + public String getVariableName() { + return BeamNode.getNodeName(this); + } + + @Override + public int getBeamNodeId() { + return beamNodeId; + } + + @Override + public void setBeamNodeId(int beamNodeId) { + this.beamNodeId = beamNodeId; + } + + @Override + public Aggregate copy(RelTraitSet traitSet, RelNode input, ImmutableBitSet groupSet, + List groupSets, List aggCalls) { + assert traitSet.containsIfApplicable(BeamConvention.INSTANCE); + final BeamAggregate newNode = + new BeamAggregate(getCluster(), traitSet, input, groupSet, groupSets, aggCalls, + aggregateWindowMinutes); + return newNode; + } + + @Override + public void toBeamStatement(List statements) { + final ParameterExpression childRecord = BeamNode.getRecordVar((BeamNode) getInput()); + + if (aggCalls.isEmpty()) { + // Handle distinct + final Expression streamKey = getDistinctStringKey(childRecord); + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), buildDistinctExpr(streamKey, childRecord))); + return; + } + + final Expression groupKey = RexBeamUtils.getStringKeyExpr(groupSet, getInput().getRowType(), childRecord); + final ParameterExpression aggregateKV = + Expressions.parameter(BeamNode.PCOLLECTION_KV_TYPE, getVariableName() + "KV"); + statements.add(Expressions.declare(0, aggregateKV, getAggregateKVExpr(groupKey, childRecord))); + + ParameterExpression groupedRecords = + Expressions.parameter(VITER_TYPE, ((BeamNode) getInput()).getVariableName() + "GroupedRecords"); + + final Expression iterator = + Expressions.call(groupedRecords, Types.lookupMethod(Iterable.class, MethodNames.ITERATOR)); + List aggStatements = getMapBodyStatements(iterator); + final Expression perKeyFunc = + RexBeamUtils.makeLambdaFunction(Types.of(SerializableFunction.class, VITER_TYPE, GenericRecord.class), + GenericRecord.class, ImmutableList.of(groupedRecords), aggStatements, new HashSet<>()); + + final Expression combineExpr = Expressions.call(Combine.class, PER_KEY, perKeyFunc); + Expression aggExpr = Expressions.call(aggregateKV, Methods.P_COLLECTION_APPLY, combineExpr); + + if (!aggCalls.isEmpty()) { + aggExpr = BeamNode.getSetSchemaCoderExpr(this, aggExpr); + } + // Reset to global window + final Expression globalWinExpr = Expressions.call(org.apache.beam.sdk.transforms.windowing.Window.class, INTO, + Expressions.new_(GlobalWindows.class)); + aggExpr = Expressions.call(aggExpr, Methods.P_COLLECTION_APPLY, globalWinExpr); + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), aggExpr)); + } + + private Expression buildDistinctExpr(Expression groupKey, ParameterExpression childRecord) { + final Expression finalExpr = resetKeyWithGroupbyCols(groupKey, childRecord); + final Expression dedupCall = Expressions.call(MessageDedup.class, WITH_IN, buildDedupDuration()); + return Expressions.call(finalExpr, Methods.P_COLLECTION_APPLY, dedupCall); + } + + private Expression buildDedupDuration() { + final CalciteBeamConfig config = ((BeamPlanner) getCluster().getPlanner())._calciteBeamConfig; + if (config.dedupTimeGranularity == Calendar.DATE) { + return Expressions.call(org.joda.time.Duration.class, STANDARD_DAYS, + Expressions.constant(config.dedupTimeValue, int.class)); + } + throw new UnsupportedOperationException( + "Calendar field of " + config.dedupTimeGranularity + " not supported"); + } + + private Expression resetKeyWithGroupbyCols(Expression groupKey, ParameterExpression childRecord) { + final ParameterExpression childExpr = BeamNode.getBeamNodeVar((BeamNode) getInput()); + final Expression valueCreate = Expressions.call(Values.class, CREATE); + final Expression finalExpr = Expressions.call(childExpr, Methods.P_COLLECTION_APPLY, valueCreate); + return RexBeamUtils.getBeamKeyFunc(finalExpr, childRecord, groupKey); + } + + private Expression getAggregateKVExpr(Expression groupKey, ParameterExpression childRecord) { + final Expression finalExpr = resetKeyWithGroupbyCols(groupKey, childRecord); + final Expression duration = Expressions.call(org.joda.time.Duration.class, STANDARD_MINUTES, + Expressions.constant(aggregateWindowMinutes, long.class)); + final Expression fixedWindow = Expressions.call(FixedWindows.class, OF, duration); + return Expressions.call(finalExpr, Methods.P_COLLECTION_APPLY, + Expressions.call(org.apache.beam.sdk.transforms.windowing.Window.class, INTO, fixedWindow)); + } + + private Expression getDistinctStringKey(Expression recordName) { + final CalciteBeamConfig config = ((BeamPlanner) getCluster().getPlanner())._calciteBeamConfig; + final Expression timestampCol = Expressions.constant(config.timestampField, String.class); + final Expression prefix = Expressions.constant(getVariableName() + "_", String.class); + return Expressions.call(BeamExecUtil.class, MethodNames.BUILD_DISTINCT_STRING_KEY_FROM_RECORD, prefix, recordName, + timestampCol); + } + + // Generate statements for body of map methods + private List getMapBodyStatements(Expression getIterator) { + if (aggCalls.isEmpty()) { + final Expression msgExpr = Expressions.call(getIterator, Types.lookupMethod(Iterator.class, MethodNames.NEXT)); + return Collections.singletonList(Expressions.return_(null, msgExpr)); + } + + final List statements = new ArrayList<>(); + final ParameterExpression aggSchema = BeamNode.getNodeSchemaVar(this); + final ParameterExpression aggRecord = BeamNode.getRecordVar(this); + + // Declare aggregate record. Code example: + // GenericRecord aggregate3Record = new GenericData.Record(AGGREGATE3_SCHEMA); + statements.add(Expressions.declare(0, aggRecord, Expressions.new_(GenericData.Record.class, aggSchema))); + + // Declare iterator. Code example: + // Iterator iterator = windowPane.getMessage()).iterator(); + final ParameterExpression iterator = + Expressions.parameter(Types.of(Iterator.class, GenericRecord.class), MethodNames.ITERATOR); + statements.add(Expressions.declare(0, iterator, getIterator)); + + // Declare input record. Code example: + // GenericRecord inputRecord = iterator.next(); + final ParameterExpression inputRecord = Expressions.parameter(GenericRecord.class, "inputRecord"); + final Expression iteratorNext = Expressions.call(iterator, MethodNames.NEXT); + statements.add(Expressions.declare(0, inputRecord, iteratorNext)); + + // Get all group columns for the aggregate records + statements.addAll(getGroupColumns(inputRecord, aggRecord)); + + // Get aggregate calls + statements.addAll(getAggregateCalls(inputRecord, aggRecord, iterator)); + + statements.add(Expressions.return_(null, aggRecord)); + return statements; + } + + // Statements to retrieve group columns + private List getGroupColumns(ParameterExpression inputRecord, ParameterExpression aggRecord) { + final RelDataType inputRowtype = getInput().getRowType(); + final List results = new ArrayList<>(); + for (int col : groupSet) { + final Expression valueExpr = + RexBeamUtils.toObjectType(RexBeamUtils.getFieldExpression(inputRowtype, col, inputRecord)); + // Then add value of the field to avro record. Code example: + // aggregate3Record.put("longGroupCol", inputRecord.get("longGroupCol")); + results.add(Expressions.statement(Expressions.call(aggRecord, MethodNames.AVRO_PUT, + BeamNode.getAvroName(inputRowtype.getFieldNames().get(col)), valueExpr))); + } + return results; + } + + // Statements to implement aggregate calls + private List getAggregateCalls(ParameterExpression inputRecord, ParameterExpression aggRecord, + Expression iterator) { + final List results = new ArrayList<>(); + + // Implementation for all aggregate calls + final List aggCallImps = new ArrayList<>(); + // Map from input column index to a pair of column variable expression and the column value expression + final Map> inputColMap = new HashMap<>(); + // Statements after while statement to save the final aggregate values into the aggregate record + final List afterWhileStmts = new ArrayList<>(); + + int aggCol = getGroupCount(); + for (AggregateCall aggCall : getAggCallList()) { + final ParameterExpression aggVar = getColumnVar(this, aggCol, true); + if (aggCall.getAggregation().getKind() == SqlKind.COLLECT) { + // Need to register schema for COLLECT + final RexToBeamConverter rexTranslator = ((BeamPlanner) getCluster().getPlanner()).getRexTranslator(); + rexTranslator.registerSchema(RexBeamUtils.getSchemaName(aggVar.name), aggCall.getType()); + } + + // Collect all input variable for aggregate calls, add input col var decl into while body. Code example: + // Long project2longCol = (Long) inputRecord.get("longCol"); + collectAggregateInput(inputRecord, aggCall, inputColMap); + + // Get agg call implementation + final AggCallImplementation callImp = AggCallImplementation.getAggCallImpl(aggCall, aggVar, inputColMap); + aggCallImps.add(callImp); + + // Declare and initialize aggregate variable. Code example: + // long aggregate3_f13 = 1; + results.add(Expressions.declare(0, aggVar, callImp.init())); + + // Store the final aggregate values to the aggregate record. Code example: + // aggregate3Record.put("_f3", (Object) aggregate3_f3); + final Expression aggResult = RexBeamUtils.toObjectType(callImp.result()); + afterWhileStmts.add(Expressions.statement(Expressions.call(aggRecord, MethodNames.AVRO_PUT, + BeamNode.getAvroName(getRowType().getFieldNames().get(aggCol)), aggResult))); + + aggCol++; + } + + // Construct the while body + final List whileBodyStmts = new ArrayList<>(); + + // Declare local variable. Code example (for COUNT): + // long aggregate3_f3 = 1; + inputColMap.values() + .stream() + .forEach(f -> whileBodyStmts.add(Expressions.declare(0, (ParameterExpression) f.get(0), f.get(1)))); + + // Add aggregate implementation to while body. Code example for COUNT: + // if (project2longCol != null) { + // aggregate3_f3 = aggregate3_f3 + 1; + // } + aggCallImps.stream().forEach(f -> whileBodyStmts.addAll(f.aggregate())); + + // Loop condition variable. Code example: + // boolean doLoop = true; + final ParameterExpression loopCond = Expressions.parameter(boolean.class, "doLoop"); + results.add(Expressions.declare(0, loopCond, Expressions.constant(true))); + + // Advance to next iterator when necessary. Code example: + // if (iterator.hasNext()) { + // inputRecord = iterator.next(); + // } else { + // doLoop = false; + // } + final Expression ifCond = Expressions.call(iterator, MethodNames.HAS_NEXT); + final Expression nextIter = Expressions.assign(inputRecord, Expressions.call(iterator, MethodNames.NEXT)); + final Expression loopEnd = Expressions.assign(loopCond, Expressions.constant(false)); + whileBodyStmts.add(Expressions.ifThenElse(ifCond, Expressions.statement(nextIter), Expressions.statement(loopEnd))); + + // Add while block to main code. Code example: + // while (doLoop) { + // // while body + // } + results.add(Expressions.while_(loopCond, Expressions.block(whileBodyStmts))); + results.addAll(afterWhileStmts); + return results; + } + + // Collects statements for getting input cols of an aggregate call + private void collectAggregateInput(ParameterExpression inputRecord, AggregateCall aggCall, + Map> inputColMap) { + final RelDataType inputRowtype = getInput().getRowType(); + for (int col : aggCall.getArgList()) { + if (!inputColMap.containsKey(col)) { + // Declare input col var. Code example: + // Long project2longCol = (Long) inputRecord.get("longCol"); + final Expression colExpr = RexBeamUtils.getFieldExpression(inputRowtype, col, inputRecord); + final Expression colVar = getColumnVar(getInput(), col, false); + inputColMap.put(col, Lists.newArrayList(colVar, colExpr)); + } + } + } + + // Gets variable expression for a column + private static ParameterExpression getColumnVar(RelNode relNode, int colIndex, boolean mutableType) { + final RelDataTypeField colField = relNode.getRowType().getFieldList().get(colIndex); + final Type colType = mutableType ? AvroJavaTypeFactory.AVRO_TYPE_FACTORY.getMutableJavaClass(colField.getType()) + : AvroJavaTypeFactory.AVRO_TYPE_FACTORY.getImmutableJavaClass(colField.getType()); + final String varName = + ((BeamNode) relNode).getVariableName() + RelDataTypeToAvro.toAvroQualifedName(colField.getName()); + return Expressions.parameter(colType, varName); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamArrayFlatten.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamArrayFlatten.java new file mode 100644 index 000000000..d93967c55 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamArrayFlatten.java @@ -0,0 +1,112 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.utils.MethodNames; +import com.linkedin.beam.utils.Methods; +import com.linkedin.beam.utils.RexBeamUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.transforms.FlatMapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.calcite.linq4j.tree.ConstantExpression; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.type.RelDataType; + +import static com.linkedin.beam.utils.MethodNames.*; + + +public class BeamArrayFlatten extends SingleRel implements BeamNode { + private final List flattenCols; + private int beamNodeId; + private final RelDataType rowType; + + /** + * Creates a BeamArrayFlatten. + * @param cluster Cluster this relational expression belongs to + * @param traits Rel trait + * @param input Input relational expression + */ + private BeamArrayFlatten(RelOptCluster cluster, RelTraitSet traits, RelNode input, List flattenCols, + RelDataType rowType) { + super(cluster, traits, input); + this.flattenCols = flattenCols; + this.rowType = rowType; + } + + public static BeamArrayFlatten create(final RelNode input, final List flattenCols, RelDataType rowType) { + final RelOptCluster cluster = input.getCluster(); + final RelTraitSet traitSet = cluster.traitSetOf(BeamConvention.INSTANCE); + return new BeamArrayFlatten(cluster, traitSet, input, flattenCols, rowType); + } + + @Override + public RelNode copy(RelTraitSet traitSet, List inputs) { + assert traitSet.containsIfApplicable(BeamConvention.INSTANCE); + final BeamArrayFlatten newRel = + new BeamArrayFlatten(getCluster(), traitSet, inputs.get(0), this.flattenCols, this.rowType); + return newRel; + } + + @Override + public String getVariableName() { + return BeamNode.getNodeName(this); + } + + @Override + public void toBeamStatement(List statements) { + final ParameterExpression inputKV = BeamNode.getBeamKVVar((BeamNode) getInput()); + final List flattenCallArgs = + ImmutableList.of(inputKV, getFlattenColList(), BeamNode.getNodeSchemaVar(this)); + final Expression flattenCallExpr = Expressions.call(com.linkedin.beam.excution.BeamArrayFlatten.class, MethodNames.FLATTEN, flattenCallArgs); + + final Expression flatMapFuncClass = + RexBeamUtils.makeLambdaFunction(Types.of(SimpleFunction.class, KV_TYPE, COLLECTION_KV_TYPE), COLLECTION_KV_TYPE, + Arrays.asList(inputKV), ImmutableList.of(Expressions.return_(null, flattenCallExpr)), null); + + final Expression viaCall = Expressions.call(FlatMapElements.class, VIA, flatMapFuncClass); + final Expression applyCall = + Expressions.call(BeamNode.getBeamNodeVar((BeamNode) getInput()), Methods.P_COLLECTION_APPLY, viaCall); + + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), + BeamNode.getSetSchemaCoderExpr(this, applyCall))); + + } + + private Expression getFlattenColList() { + final List flattenColExprs = new ArrayList<>(); + for (String flattenCol : flattenCols) { + flattenColExprs.add(BeamNode.getAvroName(flattenCol)); + } + return Expressions.call(ImmutableList.class, MethodNames.OF, flattenColExprs); + } + + @Override + public int getBeamNodeId() { + return beamNodeId; + } + + @Override + public void setBeamNodeId(int beamNodeId) { + this.beamNodeId = beamNodeId; + } + + @Override + protected RelDataType deriveRowType() { + return rowType; + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamFilter.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamFilter.java new file mode 100644 index 000000000..73b89fdf6 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamFilter.java @@ -0,0 +1,99 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.planner.BeamPlanner; +import com.linkedin.beam.utils.RexBeamUtils; +import com.linkedin.beam.utils.RexToBeamConverter; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +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.RexNode; + +import static com.linkedin.beam.utils.MethodNames.*; + + +public final class BeamFilter extends Filter implements BeamNode { + private int beamNodeId; + + private BeamFilter(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode condition) { + super(cluster, traitSet, child, condition); + } + + public static BeamFilter create(final RelNode input, RexNode condition) { + final RelOptCluster cluster = input.getCluster(); + return new BeamFilter(cluster, cluster.traitSetOf(BeamConvention.INSTANCE), input, condition); + } + + @Override + public BeamFilter copy(RelTraitSet traitSet, RelNode input, RexNode condition) { + assert traitSet.containsIfApplicable(BeamConvention.INSTANCE); + final BeamFilter newNode = new BeamFilter(getCluster(), traitSet, input, condition); + return newNode; + } + + @Override + public void toBeamStatement(List statements) { + final List filterStatements = new ArrayList<>(); + + // Get inputRecord from KVPair + filterStatements.add(BeamNode.getRecordFromKV((BeamNode) getInput())); + + final Set bodyExceptions = new HashSet<>(); + generateFilterStatements(filterStatements, bodyExceptions); + + final ParameterExpression inputKV = BeamNode.getBeamKVVar((BeamNode) getInput()); + final Expression filterFuncClass = + RexBeamUtils.makeLambdaFunction(Types.of(SerializableFunction.class, KV_TYPE, Boolean.class), Boolean.class, + Arrays.asList(inputKV), filterStatements, bodyExceptions); + + final Expression filterCall = Expressions.call(org.apache.beam.sdk.transforms.Filter.class, BY, filterFuncClass); + final Method pCollectionApplyMethod = Types.lookupMethod(PCollection.class, APPLY, PTransform.class); + final Expression applyCall = + Expressions.call(BeamNode.getBeamNodeVar((BeamNode) getInput()), pCollectionApplyMethod, filterCall); + + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), applyCall)); + } + + private void generateFilterStatements(List filterStatements, Set bodyExceptions) { + final RexToBeamConverter rexTranslator = ((BeamPlanner) getCluster().getPlanner()).getRexTranslator(); + final Expression filter = rexTranslator.convert(getCondition(), getInput()); + filterStatements.addAll(rexTranslator.getLocalVarDeclarations()); + filterStatements.add(Expressions.return_(null, filter)); + bodyExceptions.addAll(rexTranslator.getLocalExeptions()); + } + + @Override + public int getBeamNodeId() { + return beamNodeId; + } + + @Override + public void setBeamNodeId(int beamNodeId) { + this.beamNodeId = beamNodeId; + } + + @Override + public String getVariableName() { + return BeamNode.getNodeName(this); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamJoin.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamJoin.java new file mode 100644 index 000000000..d5a8afdaf --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamJoin.java @@ -0,0 +1,183 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.utils.RexBeamUtils; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.beam.sdk.schemas.utils.AvroUtils; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.CorrelationId; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rex.RexNode; + +import static com.linkedin.beam.utils.MethodNames.*; +import static com.linkedin.beam.utils.Methods.*; + + +public class BeamJoin extends Join implements BeamNode { + // Stream record ttl in minutes for STREAM_STREAM joins + // or table record ttl in days for STREAM_TABLE joins + private final long ttl; + private int beamNodeId; + private final String leftTimestampField; + private final String rightTimestampField; + private Expression leftKey = null; + private Expression rightKey = null; + + public BeamJoin(RelOptCluster cluster, RelTraitSet traitSet, RelNode left, String leftTimestampField, RelNode right, + String rightTimestampField, RexNode condition, Set variablesSet, JoinRelType joinType, long ttl) { + super(cluster, traitSet, left, right, condition, variablesSet, joinType); + this.leftTimestampField = leftTimestampField; + this.rightTimestampField = rightTimestampField; + this.ttl = ttl; + } + + public static BeamJoin create(RelNode left, String leftTimestampField, RelNode right, String rightTimestampField, + RexNode conditionExpr, Set variablesSet, JoinRelType joinType, long joinDuration) { + return new BeamJoin(left.getCluster(), left.getCluster().traitSetOf(BeamConvention.INSTANCE), left, + leftTimestampField, right, rightTimestampField, conditionExpr, variablesSet, joinType, joinDuration); + } + + @Override + public Join copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, RelNode right, JoinRelType joinType, + boolean semiJoinDone) { + assert traitSet.containsIfApplicable(BeamConvention.INSTANCE); + final BeamJoin newNode = + new BeamJoin(getCluster(), getCluster().traitSetOf(Convention.NONE), left, leftTimestampField, right, + rightTimestampField, conditionExpr, variablesSet, joinType, ttl); + return newNode; + } + + @Override + public String getVariableName() { + return BeamNode.getNodeName(this); + } + + @Override + public void toBeamStatement(List statements) { + buildKeyExprs(); + generateStreamStreamJoin(statements); + } + + private void generateStreamStreamJoin(List statements) { + final ParameterExpression leftVar = + Expressions.parameter(BeamNode.PCOLLECTION_KV_TYPE, getVariableName() + "Left"); + statements.add(Expressions.declare(0, leftVar, getBeamJoinChildExpr(getLeft(), leftKey, getLeftRecord()))); + + final ParameterExpression rightVar = + Expressions.parameter(BeamNode.PCOLLECTION_KV_TYPE, getVariableName() + "Right"); + statements.add(Expressions.declare(0, rightVar, getBeamJoinChildExpr(getRight(), rightKey, getRightRecord()))); + + Expression joinExpr = Expressions.call(org.apache.beam.sdk.schemas.transforms.Join.class, + getBeamJoinMethodName(), rightVar); + joinExpr = Expressions.call(leftVar, P_COLLECTION_APPLY, joinExpr); + + joinExpr = + BeamNode.getSetSchemaCoderExpr(this, Expressions.call(joinExpr, P_COLLECTION_APPLY, buildBeamJoinMapFunc())); + // Reset to global window + final Expression globalWinExpr = Expressions.call(Window.class, INTO, Expressions.new_(GlobalWindows.class)); + joinExpr = Expressions.call(joinExpr, P_COLLECTION_APPLY, globalWinExpr); + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), joinExpr)); + } + + private String getBeamJoinMethodName() { + switch (joinType) { + case FULL: + return "fullOuterJoin"; + case LEFT: + return "leftOuterJoin"; + case RIGHT: + return "rightOuterJoin"; + default: + return "innerJoin"; + } + } + + private Expression applyBeamJoinKey(RelNode child, Expression joinKey, ParameterExpression childRecord) { + final Expression valueCreate = Expressions.call(Values.class, CREATE); + final Expression finalExpr = + Expressions.call(BeamNode.getBeamNodeVar((BeamNode) child), P_COLLECTION_APPLY, valueCreate); + return RexBeamUtils.getBeamKeyFunc(finalExpr, childRecord, joinKey); + } + + private Expression getBeamJoinChildExpr(RelNode child, Expression joinKey, ParameterExpression childRecord) { + final Expression keyExpr = applyBeamJoinKey(child, joinKey, childRecord); + final Expression duration = + Expressions.call(org.joda.time.Duration.class, STANDARD_MINUTES, Expressions.constant(ttl, long.class)); + final Expression fixedWindow = Expressions.call(FixedWindows.class, OF, duration); + return Expressions.call(keyExpr, P_COLLECTION_APPLY, Expressions.call(Window.class, INTO, fixedWindow)); + } + +private Expression buildBeamJoinMapFunc() { + final ParameterExpression rowVar = Expressions.parameter(Row.class, getVariableName() + "Row"); + final List joinStatements = new ArrayList<>(); + joinStatements.add(Expressions.declare(0, getJoinedRecord(), + Expressions.call(AvroUtils.class, "toGenericRecord", rowVar, BeamNode.getNodeSchemaVar(this)))); + final ParameterExpression keyVar = Expressions.parameter(String.class, "key"); + joinStatements.add(Expressions.declare(0, keyVar, + RexBeamUtils.getStringKeyExpr(joinInfo.leftKeys, getRowType(), getJoinedRecord()))); + + joinStatements.add(Expressions.return_(null, Expressions.call(KV.class, OF, + keyVar, getJoinedRecord()))); + final Expression joinMapExpr = + RexBeamUtils.makeLambdaFunction(Types.of(SimpleFunction.class, Row.class, KV_TYPE), KV_TYPE, + Collections.singletonList(rowVar), joinStatements, new HashSet<>()); + return Expressions.call(MapElements.class, VIA, joinMapExpr); +} + + private void buildKeyExprs() { + if (leftKey == null || rightKey == null) { + final JoinInfo joinInfo = analyzeCondition(); + leftKey = RexBeamUtils.getStringKeyExpr(joinInfo.leftKeys, getLeft().getRowType(), getLeftRecord()); + rightKey = RexBeamUtils.getStringKeyExpr(joinInfo.rightKeys, getRight().getRowType(), getRightRecord()); + } + } + + private ParameterExpression getLeftRecord() { + return BeamNode.getRecordVar((BeamNode) getLeft()); + } + + private ParameterExpression getRightRecord() { + return BeamNode.getRecordVar((BeamNode) getRight()); + } + + private ParameterExpression getJoinedRecord() { + return BeamNode.getRecordVar(this); + } + + @Override + public int getBeamNodeId() { + return beamNodeId; + } + + @Override + public void setBeamNodeId(int beamNodeId) { + this.beamNodeId = beamNodeId; + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamNode.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamNode.java new file mode 100644 index 000000000..7d4451952 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamNode.java @@ -0,0 +1,89 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.utils.RelDataTypeToAvro; +import com.linkedin.beam.utils.RexBeamUtils; +import java.lang.reflect.Modifier; +import java.lang.reflect.Type; +import java.util.Collection; +import java.util.List; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.linq4j.tree.ConstantExpression; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.rel.RelNode; + +import static com.linkedin.beam.utils.MethodNames.*; +import static com.linkedin.beam.utils.Methods.*; + + +public interface BeamNode { + Type KV_TYPE = Types.of(KV.class, String.class, GenericRecord.class); + Type VITER_TYPE = Types.of(Iterable.class, GenericRecord.class); + Type PCOLLECTION_KV_TYPE = Types.of(PCollection.class, KV_TYPE); + Type COLLECTION_KV_TYPE = Types.of(Collection.class, KV_TYPE); + + Expression STRING_ENCODER = Expressions.call(StringUtf8Coder.class, OF); + + String getVariableName(); + + /** + * Converts the Beam node into Java Beam API statements. + * @param statements List to contain all result statements + */ + void toBeamStatement(List statements); + + int getBeamNodeId(); + + void setBeamNodeId(int beamNodeId); + + static String getNodeName(RelNode relNode) { + return relNode.getRelTypeName().toLowerCase().replace("beam", "") + ((BeamNode) relNode).getBeamNodeId(); + } + + static ParameterExpression getBeamNodeVar(BeamNode node) { + return Expressions.parameter(PCOLLECTION_KV_TYPE, node.getVariableName()); + } + + static ParameterExpression getNodeSchemaVar(BeamNode node) { + return Expressions.parameter(Schema.class, RexBeamUtils.getSchemaName(node.getVariableName())); + } + + static ParameterExpression getRecordVar(BeamNode beamNode) { + return Expressions.parameter(GenericRecord.class, beamNode.getVariableName() + "Record"); + } + + static ParameterExpression getBeamKVVar(BeamNode beamNode) { + return Expressions.parameter(KV_TYPE, beamNode.getVariableName() + "KV"); + } + + static ConstantExpression getAvroName(String relFieldName) { + return Expressions.constant(RelDataTypeToAvro.toAvroQualifedName(relFieldName), String.class); + } + + static Statement getRecordFromKV(BeamNode beamNode) { + final ParameterExpression recordVar = getRecordVar(beamNode); + final ParameterExpression kvVar = getBeamKVVar(beamNode); + return Expressions.declare(Modifier.FINAL, recordVar, Expressions.call(kvVar, GET_VALUE, ImmutableList.of())); + } + + static Expression getSetSchemaCoderExpr(BeamNode beamNode, Expression receiver) { + final Expression schemaCoder = Expressions.call(AvroCoder.class, OF, getNodeSchemaVar(beamNode)); + final Expression kvEncoder = Expressions.call(KvCoder.class, OF, STRING_ENCODER, schemaCoder); + return Expressions.call(receiver, P_COLLECTION_SET_CODER, kvEncoder); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamProject.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamProject.java new file mode 100644 index 000000000..b8e5f9c5d --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamProject.java @@ -0,0 +1,124 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.planner.BeamPlanner; +import com.linkedin.beam.utils.Methods; +import com.linkedin.beam.utils.RexBeamUtils; +import com.linkedin.beam.utils.RexToBeamConverter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.avro.generic.GenericData; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +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.RexNode; + +import static com.linkedin.beam.utils.MethodNames.*; + + +public class BeamProject extends Project implements BeamNode { + private int beamNodeId; + + private BeamProject(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) { + final BeamProject newNode = new BeamProject(getCluster(), traitSet, input, exps, rowType); + return newNode; + } + + public static BeamProject create(final RelNode input, final List projects, RelDataType rowType) { + final RelOptCluster cluster = input.getCluster(); + return new BeamProject(cluster, cluster.traitSet().replace(BeamConvention.INSTANCE), input, projects, rowType); + } + + @Override + public String getVariableName() { + return BeamNode.getNodeName(this); + } + + @Override + public void toBeamStatement(List statements) { + final List projectStatements = new ArrayList<>(); + + // Get inputRecord from KVPair + projectStatements.add(BeamNode.getRecordFromKV((BeamNode) getInput())); + + final Set bodyExceptions = new HashSet<>(); + generateProjectStatements(projectStatements, bodyExceptions); + // Add the final return statement + final ParameterExpression inputKV = BeamNode.getBeamKVVar((BeamNode) getInput()); + projectStatements.add(Expressions.return_(null, Expressions.call(KV.class, OF, + ImmutableList.of(Expressions.call(inputKV, GET_KEY), BeamNode.getRecordVar(this))))); + + final Expression mapFuncClass = + RexBeamUtils.makeLambdaFunction(Types.of(SimpleFunction.class, KV_TYPE, KV_TYPE), KV_TYPE, + Arrays.asList(inputKV), projectStatements, bodyExceptions); + + final Expression viaCall = Expressions.call(MapElements.class, VIA, mapFuncClass); + final Expression applyCall = + Expressions.call(BeamNode.getBeamNodeVar((BeamNode) getInput()), Methods.P_COLLECTION_APPLY, viaCall); + + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), + BeamNode.getSetSchemaCoderExpr(this, applyCall))); + } + + private void generateProjectStatements(List projectStatements, Set bodyExceptions) { + final RexToBeamConverter rexTranslator = ((BeamPlanner) getCluster().getPlanner()).getRexTranslator(); + final ParameterExpression projectedSchema = BeamNode.getNodeSchemaVar(this); + final ParameterExpression projectedRecord = BeamNode.getRecordVar(this); + projectStatements.add( + Expressions.declare(0, projectedRecord, Expressions.new_(GenericData.Record.class, projectedSchema))); + + // Do projection + final List fieldNames = getRowType().getFieldNames(); + final List projects = getProjects(); + for (int i = 0; i < projects.size(); i++) { + final Expression projection = rexTranslator.convert(projects.get(i), getInput(), getFieldSchemaName(i)); + // Add local var declaration statements first + projectStatements.addAll(rexTranslator.getLocalVarDeclarations()); + + // Then add value of the field to avro record + projectStatements.add(Expressions.statement( + Expressions.call(projectedRecord, Methods.AVRO_PUT, BeamNode.getAvroName(fieldNames.get(i)), projection))); + // Capture exceptions if any + bodyExceptions.addAll(rexTranslator.getLocalExeptions()); + } + } + + private String getFieldSchemaName(int i) { + return getVariableName().toUpperCase() + "_COLUMN" + i + "_SCHEMA"; + } + + @Override + public int getBeamNodeId() { + return beamNodeId; + } + + @Override + public void setBeamNodeId(int beamNodeId) { + this.beamNodeId = beamNodeId; + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamTableScan.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamTableScan.java new file mode 100644 index 000000000..b56a04be7 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamTableScan.java @@ -0,0 +1,133 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.planner.BeamCodeGenerator; +import com.linkedin.beam.excution.BeamAPIUtil; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import com.linkedin.beam.planner.CalciteBeamConfig; +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.planner.BeamPlanner; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +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.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.sql.type.SqlTypeName; + +import static com.linkedin.beam.utils.MethodNames.*; + + +public final class BeamTableScan extends TableScan implements BeamNode { + private static final String SCAN_VARIABLE_NAME = "stream"; + private int beamNodeId; + private CalciteBeamConfig.StreamConfig streamConfig = null; + + private BeamTableScan(RelOptCluster cluster, RelTraitSet traitSet, RelOptTable table) { + super(cluster, traitSet, table); + } + + public static BeamTableScan create(RelOptCluster cluster, final RelOptTable relOptTable) { + final RelTraitSet traitSet = cluster.traitSetOf(BeamConvention.INSTANCE); + return new BeamTableScan(cluster, traitSet, relOptTable); + } + + @Override + public String getVariableName() { + return SCAN_VARIABLE_NAME + getBeamNodeId(); + } + + @Override + public int getBeamNodeId() { + return beamNodeId; + } + + @Override + public void setBeamNodeId(int beamNodeId) { + this.beamNodeId = beamNodeId; + } + + @Override + public void toBeamStatement(List statements) { + final Expression readGenericRecord = Expressions.call(KafkaIOGenericRecord.class, READ); + Expression withTopic = + Expressions.call(readGenericRecord, WITH_TOPIC, Expressions.constant(getInputStreamName(), String.class)); + + final List timeField = getStreamConfig().timeField; + if (timeField != null && !timeField.isEmpty()) { + validateTimeSchema(timeField); + final List timeColumns = new ArrayList<>(); + for (String column : timeField) { + timeColumns.add(Expressions.constant(column, String.class)); + } + final Expression timeColumnList = Expressions.call(ImmutableList.class, OF, timeColumns); + final Expression timeFn = Expressions.call(BeamAPIUtil.class, WITH_TIME_FUNCTION, timeColumnList); + withTopic = Expressions.call(withTopic, WITH_TIMESTAMP_FN, timeFn); + } + + final Expression withoutMetadata = Expressions.call(withTopic, WITHOUT_METADATA); + + final Method pipelineApplyMethod = Types.lookupMethod(Pipeline.class, APPLY, String.class, PTransform.class); + final Expression tableScanExpr = Expressions.call(BeamCodeGenerator.PIPELINE, pipelineApplyMethod, ImmutableList.of( + Expressions.constant("Read " + getInputStreamName(), String.class), withoutMetadata)); + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), tableScanExpr)); + } + + private void validateTimeSchema(List timeField) { + RelDataType recordType = getRowType(); + for (int i = 0; i < timeField.size(); i++) { + final RelDataTypeField field = recordType.getField(timeField.get(i), true, false); + if (field == null) { + fail(timeField); + } + if (i < timeField.size() - 1) { + if (!field.getType().isStruct()) { + fail(timeField); + } + recordType = field.getType(); + continue; + } + + // Final time field should be bigint + if (field.getType().getSqlTypeName() != SqlTypeName.BIGINT) { + throw new IllegalArgumentException("Invalid type for time column '" + field.getName() + "'. Expect BIGINT (long) type," + + " actual type: " + field.getType().getSqlTypeName()); + } + } + } + + private void fail(List timeField) { + throw new IllegalArgumentException("Invalid schema for time field. Time field: " + timeField + + ". Record schema: " + getRowType()); + } + + + private CalciteBeamConfig.StreamConfig getStreamConfig() { + if (streamConfig == null) { + streamConfig = getCalciteBeamConfig().getInputStream(getTable().getQualifiedName()); + } + return streamConfig; + } + + private CalciteBeamConfig getCalciteBeamConfig() { + return ((BeamPlanner) getCluster().getPlanner())._calciteBeamConfig; + } + + public String getInputStreamName() { + return getStreamConfig().topic; + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/operators/BeamUnion.java b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamUnion.java new file mode 100644 index 000000000..fb98ba664 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/operators/BeamUnion.java @@ -0,0 +1,78 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.operators; + +import com.linkedin.beam.planner.BeamConvention; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Union; + +import static com.linkedin.beam.utils.MethodNames.*; + + +public class BeamUnion extends Union implements BeamNode { + private int beamNodeId; + + private BeamUnion(RelOptCluster cluster, RelTraitSet traits, List inputs, boolean all) { + super(cluster, traits, inputs, all); + } + + public static BeamUnion create(List inputs, boolean all) { + assert inputs.size() > 0; + final RelOptCluster cluster = inputs.get(0).getCluster(); + return new BeamUnion(cluster, cluster.traitSetOf(BeamConvention.INSTANCE), inputs, all); + } + + @Override + public BeamUnion copy(RelTraitSet traitSet, List inputs, boolean all) { + final BeamUnion newNode = new BeamUnion(getCluster(), traitSet, inputs, all); + return newNode; + } + + @Override + public String getVariableName() { + return BeamNode.getNodeName(this); + } + + @Override + public void toBeamStatement(List statements) { + final List childVars = new ArrayList<>(); + for (RelNode relNode : getInputs()) { + childVars.add(BeamNode.getBeamNodeVar((BeamNode) relNode)); + } + Expression listExpr = Expressions.call(PCollectionList.class, OF, childVars.get(0)); + for (int i = 1; i < childVars.size(); i++) { + listExpr = Expressions.call(listExpr, AND, childVars.get(i)); + } + + final Method pCollectionApplyMethod = Types.lookupMethod(PCollectionList.class, APPLY, PTransform.class); + final Expression merge = Expressions.call(Flatten.class, P_COLLECTIONS); + + statements.add(Expressions.declare(0, BeamNode.getBeamNodeVar(this), + Expressions.call(listExpr, pCollectionApplyMethod, merge))); + } + + @Override + public int getBeamNodeId() { + return beamNodeId; + } + + @Override + public void setBeamNodeId(int beamNodeId) { + this.beamNodeId = beamNodeId; + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/BeamCodeGenerator.java b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamCodeGenerator.java new file mode 100644 index 000000000..ef4d68dcd --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamCodeGenerator.java @@ -0,0 +1,114 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import com.linkedin.beam.operators.BeamNode; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.calcite.linq4j.tree.BlockBuilder; +import org.apache.calcite.linq4j.tree.ClassDeclaration; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.MemberDeclaration; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.rel.RelNode; + +import static com.linkedin.beam.utils.MethodNames.*; + + +/** + * Code generator for Beam APIs. + */ +public class BeamCodeGenerator extends CodeGenerator { + private static final ParameterExpression ARGS = Expressions.parameter(String[].class, "args"); + public static final ParameterExpression PIPELINE_OPTIONS = + Expressions.parameter(PipelineOptions.class, "pipelineOpts"); + public static final ParameterExpression PIPELINE = Expressions.parameter(Pipeline.class, "pipeline"); + + protected BeamCodeGenerator(CalciteBeamConfig calciteBeamConfig) { + super(calciteBeamConfig); + } + + @Override + protected String toJavaCode(RelNode beamPlan) { + final List initVarDeclarations = new ArrayList<>(); + + final List mainMethodStmts = new ArrayList<>(); + final Statement initStaticVarsStmt = initStaticVars(initVarDeclarations); + if (initStaticVarsStmt != null) { + mainMethodStmts.add(initStaticVarsStmt); + } + + final List classDeclarations = new ArrayList<>(); + + final MemberDeclaration mainMethod = generateMainMethod(beamPlan, mainMethodStmts); + + // FIRST: declare all required schema constants + classDeclarations.addAll(declareSchemas(beamPlan)); + + // SECOND: declare other static variables + classDeclarations.addAll(initVarDeclarations); + + // THIRD: declare the main() method + classDeclarations.add(mainMethod); + + final ClassDeclaration classDeclaration = + Expressions.classDecl(Modifier.PUBLIC, _calciteBeamConfig.applicationName, null, + ImmutableList.of(), classDeclarations); + + return Expressions.toString(classDeclaration); + } + + @Override + protected void beamNodeToStatement(BeamNode beamNode, List statements) { + beamNode.toBeamStatement(statements); + } + + private MemberDeclaration generateMainMethod(RelNode beamPlan, List stmts) { + // PipelineOption + stmts.add(Expressions.declare(Modifier.FINAL, PIPELINE_OPTIONS, + Expressions.call(PipelineOptionsFactory.class, CREATE))); + + // Pipeline + stmts.add(Expressions.declare(Modifier.FINAL, PIPELINE, + Expressions.call(Pipeline.class, CREATE, ImmutableList.of(PIPELINE_OPTIONS)))); + + // Convert all Beam nodes into Java code + relNodeToJava(beamPlan, stmts); + + // Add the final write + stmts.add(addFinalWrite(beamPlan)); + + // run + stmts.add(Expressions.statement(Expressions.call(PIPELINE, RUN))); + + final BlockBuilder body = new BlockBuilder(false); + stmts.forEach(body::add); + + return Expressions.methodDecl(Modifier.PUBLIC | Modifier.STATIC, void.class, MAIN, + ImmutableList.of(ARGS), body.toBlock()); + } + + private Statement addFinalWrite(RelNode beamPlan) { + final Expression rootExpr = BeamNode.getBeamNodeVar((BeamNode) beamPlan); + final Expression writeRecord = Expressions.call(KafkaIOGenericRecord.class, WRITE); + final Expression withTopic = Expressions.call(writeRecord, WITH_TOPIC, + Expressions.constant(_calciteBeamConfig.kafkaOutputTopic, String.class)); + final Method pipelineApplyMethod = Types.lookupMethod(Pipeline.class, APPLY, String.class, PTransform.class); + final Expression writeKafka = Expressions.call(rootExpr, pipelineApplyMethod, withTopic); + return Expressions.statement(writeKafka); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/BeamConvention.java b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamConvention.java new file mode 100644 index 000000000..75bc3a6a7 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamConvention.java @@ -0,0 +1,51 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import com.linkedin.beam.operators.BeamNode; +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 BeamConvention implements Convention { + INSTANCE; + + @Override + public String toString() { + return getName(); + } + + public Class getInterface() { + return BeamNode.class; + } + + public String getName() { + return "BEAM"; + } + + public RelTraitDef getTraitDef() { + return ConventionTraitDef.INSTANCE; + } + + public boolean satisfies(RelTrait trait) { + return this == trait; + } + + public void register(RelOptPlanner planner) { + } + + public boolean canConvertConvention(Convention toConvention) { + return false; + } + + public boolean useAbstractConvertersForConversion(RelTraitSet fromTraits, RelTraitSet toTraits) { + return false; + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/BeamPlanner.java b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamPlanner.java new file mode 100644 index 000000000..e6e4d55a7 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamPlanner.java @@ -0,0 +1,232 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.operators.BeamNode; +import com.linkedin.beam.utils.RexToBeamConverter; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.calcite.piglet.PigToSqlAggregateRule; +import org.apache.calcite.piglet.PigUserDefinedFunction; +import org.apache.calcite.plan.Context; +import org.apache.calcite.plan.ConventionTraitDef; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptCost; +import org.apache.calcite.plan.RelOptCostFactory; +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.volcano.VolcanoPlanner; +import org.apache.calcite.rel.BiRel; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelCollationTraitDef; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Correlate; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.core.Uncollect; +import org.apache.calcite.rel.core.Union; +import org.apache.calcite.rel.core.Values; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.logical.LogicalCorrelate; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.rel.logical.LogicalTableScan; +import org.apache.calcite.rel.logical.LogicalUnion; +import org.apache.calcite.rel.logical.LogicalValues; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.rel.rules.FilterMergeRule; +import org.apache.calcite.rel.rules.FilterProjectTransposeRule; +import org.apache.calcite.rel.rules.ProjectMergeRule; +import org.apache.calcite.tools.Program; +import org.apache.calcite.tools.Programs; +import org.apache.calcite.tools.RuleSets; + + +public class BeamPlanner extends VolcanoPlanner { + static final List IMPLEMENTATION_RULES = + ImmutableList.of( + PigToSqlAggregateRule.INSTANCE, + BeamRules.BEAM_TABLE_SCAN_RULE, + BeamRules.BEAM_FILTER_RULE, + BeamRules.BEAM_PROJECT_RULE, + BeamRules.BEAM_JOIN_RULE, + BeamRules.BEAM_UNION_RULE, + BeamRules.BEAM_AGGREGATE_RULE, + BeamRules.BEAM_ARRAY_FLATTEN_RULE); + + // Add a new transformation rule here only if you know what the rule does. + static final List TRANSFORMATION_RULES = + ImmutableList.of( + FilterMergeRule.INSTANCE, + ProjectMergeRule.INSTANCE, + FilterProjectTransposeRule.INSTANCE); + + private int beamNodeId = 0; + private final Map nodeIdMap = new HashMap<>(); + public final CalciteBeamConfig _calciteBeamConfig; + private final RexToBeamConverter rexTranslator; + + public BeamPlanner(RelOptCostFactory costFactory, Context externalContext, + CalciteBeamConfig calciteBeamConfig, Map pigUDFs, boolean planOptimized) { + super(costFactory, externalContext); + this._calciteBeamConfig = calciteBeamConfig; + rexTranslator = new RexToBeamConverter(pigUDFs); + for (RelOptRule rule : IMPLEMENTATION_RULES) { + addRule(rule); + } + if (planOptimized) { + for (RelOptRule rule : TRANSFORMATION_RULES) { + addRule(rule); + } + } + addRelTraitDef(ConventionTraitDef.INSTANCE); + addRelTraitDef(RelCollationTraitDef.INSTANCE); + } + + private int getBeamNodeId(int relId) { + Integer nodeId = nodeIdMap.get(relId); + if (nodeId == null) { + nodeId = ++beamNodeId; + nodeIdMap.putIfAbsent(relId, nodeId); + } + return nodeId; + } + + /** + * Returns a {@link RexToBeamConverter}. This method will also reset the local variable counter. + * So do not call it multiple times for the same scope of generated code. + * + * @return a {@link RexToBeamConverter}. + */ + public RexToBeamConverter getRexTranslator() { + rexTranslator.resetVarCounter(); + return rexTranslator; + } + + /** + * Creates a {@link BeamPlanner} from a @{@link RelOptPlanner} template. + * + * @param template RelOptPlanner template + * @param calciteBeamConfig Beam app config + * @param pigUDFs Pig UDF map + * @param planOptimization Whether to optimize the Calcite plan + * @return A {@link BeamPlanner} + */ + public static BeamPlanner createPlanner(RelOptPlanner template, CalciteBeamConfig calciteBeamConfig, + Map pigUDFs, boolean planOptimization) { + return new BeamPlanner(template.getCostFactory(), template.getContext(), calciteBeamConfig, pigUDFs, + planOptimization); + } + + @Override + public RelOptCost getCost(RelNode rel, RelMetadataQuery mq) { + RelOptCost cost = super.getCost(rel, mq); + if (rel instanceof Aggregate) { + final Aggregate agg = (Aggregate) rel; + if (agg.getAggCallList().size() == 1) { + AggregateCall aggCall = agg.getAggCallList().get(0); + // Make Pig aggregates 10 times more expensive to have the @PigToSqlAggregateRule applied. + if (aggCall.getAggregation().getName().equals("COLLECT")) { + cost = costFactory.makeCost(10 * cost.getRows(), 10 * cost.getCpu(), 10 * cost.getIo()); + } + } + } + return cost; + } + + + + public RelNode toBeamPlan(RelNode logicalPlan) { + final Program program = Programs.of(RuleSets.ofList(ruleSet)); + final RelNode resetPlan = resetPlanner(logicalPlan); + final RelCollation collation = resetPlan instanceof Sort ? ((Sort) resetPlan).collation : RelCollations.EMPTY; + final RelNode physicalPlan = program.run(this, resetPlan, + resetPlan.getTraitSet().replace(BeamConvention.INSTANCE).replace(collation).simplify(), + ImmutableList.of(), ImmutableList.of()); + setNodeIds(physicalPlan); + return physicalPlan; + } + + private void setNodeIds(RelNode relNode) { + for (RelNode relNode1: relNode.getInputs()) { + setNodeIds(relNode1); + } + final BeamNode beamNode = (BeamNode) relNode; + beamNode.setBeamNodeId(this.getBeamNodeId(relNode.getId())); + } + + private RelNode resetPlanner(RelNode root) { + final RelOptCluster newCluster = RelOptCluster.create(this, root.getCluster().getRexBuilder()); + return copyPlan(root, newCluster); + } + + private RelNode copyPlan(RelNode relNode, RelOptCluster newCluster) { + if (relNode instanceof TableScan) { + return LogicalTableScan.create(newCluster, relNode.getTable()); + } + + if (relNode instanceof Values) { + return LogicalValues.create(newCluster, relNode.getRowType(), ((Values) relNode).tuples); + } + + if (relNode instanceof SingleRel) { + final RelNode inputRel = copyPlan(((SingleRel) relNode).getInput(), newCluster); + if (relNode instanceof Filter) { + return LogicalFilter.create(inputRel, ((Filter) relNode).getCondition()); + } + if (relNode instanceof Project) { + final Project project = (Project) relNode; + return LogicalProject.create(inputRel, project.getProjects(), project.getRowType()); + } + if (relNode instanceof Aggregate) { + final Aggregate aggregate = (Aggregate) relNode; + return LogicalAggregate.create( + inputRel, aggregate.getGroupSet(), aggregate.getGroupSets(), aggregate.getAggCallList()); + } + if (relNode instanceof Uncollect) { + final Uncollect uncollect = (Uncollect) relNode; + return Uncollect.create(uncollect.getTraitSet(), inputRel, uncollect.withOrdinality); + } + } + + if (relNode instanceof BiRel) { + final RelNode left = copyPlan(((BiRel) relNode).getLeft(), newCluster); + final RelNode right = copyPlan(((BiRel) relNode).getRight(), newCluster); + if (relNode instanceof Join) { + final Join join = (Join) relNode; + return LogicalJoin.create(left, right, join.getCondition(), join.getVariablesSet(), join.getJoinType()); + } + if (relNode instanceof Correlate) { + final Correlate correl = (Correlate) relNode; + return LogicalCorrelate.create( + left, right, correl.getCorrelationId(), correl.getRequiredColumns(), correl.getJoinType()); + } + } + + if (relNode instanceof Union) { + final Union union = (Union) relNode; + final List newInputs = new ArrayList<>(); + for (RelNode child : union.getInputs()) { + newInputs.add(copyPlan(child, newCluster)); + } + return LogicalUnion.create(newInputs, union.all); + } + + throw new UnsupportedOperationException( + relNode.getRelTypeName() + " is not supported for being translated into Beam code"); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/BeamRules.java b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamRules.java new file mode 100644 index 000000000..bf1740267 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/BeamRules.java @@ -0,0 +1,29 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import com.linkedin.beam.planner.rules.BeamAggregateRule; +import com.linkedin.beam.planner.rules.BeamArrayFlattenRule; +import com.linkedin.beam.planner.rules.BeamFilterRule; +import com.linkedin.beam.planner.rules.BeamJoinRule; +import com.linkedin.beam.planner.rules.BeamProjectRule; +import com.linkedin.beam.planner.rules.BeamTableScanRule; +import com.linkedin.beam.planner.rules.BeamUnionRule; +import org.apache.calcite.plan.RelOptRule; + + +public class BeamRules { + private BeamRules() { + } + + public static final RelOptRule BEAM_TABLE_SCAN_RULE = new BeamTableScanRule(); + public static final RelOptRule BEAM_FILTER_RULE = new BeamFilterRule(); + public static final RelOptRule BEAM_PROJECT_RULE = new BeamProjectRule(); + public static final RelOptRule BEAM_JOIN_RULE = new BeamJoinRule(); + public static final RelOptRule BEAM_UNION_RULE = new BeamUnionRule(); + public static final RelOptRule BEAM_AGGREGATE_RULE = new BeamAggregateRule(); + public static final RelOptRule BEAM_ARRAY_FLATTEN_RULE = new BeamArrayFlattenRule(); +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/CalciteBeamConfig.java b/coral-beam/src/main/java/com/linkedin/beam/planner/CalciteBeamConfig.java new file mode 100644 index 000000000..f1b154d93 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/CalciteBeamConfig.java @@ -0,0 +1,66 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import java.util.ArrayList; +import java.util.Calendar; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +/** + * Configuration for Beam physical operators, obtained from dataset.conf + */ +public class CalciteBeamConfig { + public static final long DEFAULT_AGGREGATE_WINDOW_MINUTES = 5L; + public static final long DEFAULT_JOIN_WINDOW_MINUTES = 5L; + public static final int DEFAULT_DEDUP_TIME_GRANULARITY = Calendar.DATE; + public static final int DEFAULT_DEDUP_TIME_VALUE = 1; + + public static class StreamConfig { + public final String topic; + public final List timeField; + + public StreamConfig(String topic) { + this(topic, new ArrayList<>()); + } + + public StreamConfig(String topic, List timeField) { + this.topic = topic; + this.timeField = timeField; + } + } + + public final String applicationName; + public final String timestampField; + public final String kafkaOutputTopic; + // Map from table names to Kafka input streams + public final Map tableToInputStreams = new HashMap<>(); + public long joinWindowMinutes = DEFAULT_JOIN_WINDOW_MINUTES; + public long aggregateWindowMinutes = DEFAULT_AGGREGATE_WINDOW_MINUTES; + + public int dedupTimeGranularity = DEFAULT_DEDUP_TIME_GRANULARITY; + public int dedupTimeValue = DEFAULT_DEDUP_TIME_VALUE; + + public boolean planOptimized = false; + + public CalciteBeamConfig(String applicationName, String timestampField, String kafkaOutputTopic) { + this.applicationName = applicationName; + this.timestampField = timestampField; + this.kafkaOutputTopic = kafkaOutputTopic; + } + + public StreamConfig getInputStream(List tableFullNames) { + final String tableName = tableFullNames.get(tableFullNames.size() - 1); + final StreamConfig inputStream = tableToInputStreams.get(tableName); + if (inputStream != null) { + return inputStream; + } + + throw new IllegalArgumentException("No input stream specified for the table: " + tableFullNames); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/CodeGenerator.java b/coral-beam/src/main/java/com/linkedin/beam/planner/CodeGenerator.java new file mode 100644 index 000000000..df38ac116 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/CodeGenerator.java @@ -0,0 +1,366 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import com.google.common.collect.Sets; +import com.google.googlejavaformat.java.Formatter; +import com.google.googlejavaformat.java.FormatterException; +import com.linkedin.beam.excution.BeamExecUtil; +import com.linkedin.beam.operators.BeamArrayFlatten; +import com.linkedin.beam.operators.BeamJoin; +import com.linkedin.beam.operators.BeamNode; +import com.linkedin.beam.utils.PigUDFUtils; +import com.linkedin.beam.utils.RelDataTypeToAvro; +import com.linkedin.beam.utils.RexBeamUtils; +import com.linkedin.beam.utils.RexToBeamConverter; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.avro.Schema; +import org.apache.calcite.linq4j.tree.ConstantExpression; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.FieldDeclaration; +import org.apache.calcite.linq4j.tree.MemberDeclaration; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.piglet.PigConverter; +import org.apache.calcite.piglet.PigUserDefinedFunction; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.type.DynamicRecordType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.schema.impl.ScalarFunctionImpl; +import org.apache.calcite.tools.Frameworks; +import org.apache.pig.builtin.RANDOM; +import org.apache.pig.scripting.jython.JythonFunction; + +import static com.linkedin.beam.utils.MethodNames.*; + + +/** + * Code generator to generate Beam Java code from Calcite plans. + */ +@SuppressWarnings("ALL") +public abstract class CodeGenerator { + private static final Set EXCLUDED_PIG_UDFS = Sets.newHashSet(RANDOM.class); + protected final CalciteBeamConfig _calciteBeamConfig; + protected final Map pigUdfMap; + private final Set visitedNodes; + private int recordIndex = 0; + + protected CodeGenerator(CalciteBeamConfig calciteBeamConfig) { + this._calciteBeamConfig = calciteBeamConfig; + pigUdfMap = new HashMap<>(); + visitedNodes = new HashSet<>(); + } + + public String generateJavaCodeFromPigFile(String pigFile, Map params) throws Exception { + return generateJavaCodeFromRelPlan(getRelNodeFromPigFile(pigFile, params)); + } + + public String generateJavaCodeFromPigQuery(String pigFile) throws Exception { + return generateJavaCodeFromRelPlan(getRelNodeFromPigQuery(pigFile)); + } + + public RelNode getBeamPhysicalPlan(RelNode rootNode) { + final BeamPlanner planner = BeamPlanner.createPlanner( + rootNode.getCluster().getPlanner(), _calciteBeamConfig, pigUdfMap, _calciteBeamConfig.planOptimized); + final RelNode physicalPlan = planner.toBeamPlan(rootNode); + + pigUdfMap.clear(); + PigUDFUtils.collectPigUDFs(physicalPlan, pigUdfMap); + return physicalPlan; + } + + public String generateJavaCodeFromRelPlan(RelNode rootNode) { + return generateJavaCodeFromBeamPhysicalPlan(getBeamPhysicalPlan(rootNode)); + } + + public String generateJavaCodeFromBeamPhysicalPlan(RelNode beamPlan) { + final String rawCode = fixVarName(toJavaCode(beamPlan)); + final Formatter formatter = new Formatter(); + try { + return formatter.formatSource(extractPathForImport(rawCode)); + } catch (FormatterException e) { + throw new IllegalArgumentException("Unable to parse auto-gen code", e); + } + } + + /** + * Declares all required schema constants. + */ + List declareSchemas(RelNode relNode) { + final List results = new ArrayList<>(); + final Map schemaMaps = new LinkedHashMap<>(); + + // Get schemas for all required relational operators + collectRelOpSchemas(relNode, schemaMaps); + + // Get schemas for all required relational expressions + final RexToBeamConverter rexTranslator = ((BeamPlanner) relNode.getCluster().getPlanner()).getRexTranslator(); + schemaMaps.putAll(rexTranslator.getRegisteredSchemas()); + + // Now declare those schemas + for (Map.Entry schema : schemaMaps.entrySet()) { + results.add(makeSchemaDeclaration(RexBeamUtils.getSchemaParam(schema.getKey()), schema.getValue())); + } + + return results; + } + + /** + * Collects all required schemas that are used in the code. + * + * @param relNode Root node + * @param schemaMaps Map to store required schemas + */ + private void collectRelOpSchemas(RelNode relNode, Map schemaMaps) { + for (RelNode relNode1 : relNode.getInputs()) { + collectRelOpSchemas(relNode1, schemaMaps); + } + if (relNode instanceof Project + || relNode instanceof BeamArrayFlatten + || relNode instanceof Join + || relNode instanceof Aggregate && !((Aggregate) relNode).getAggCallList().isEmpty()) { + schemaMaps.put(RexBeamUtils.getSchemaName(((BeamNode) relNode).getVariableName()), relNode.getRowType()); + if (relNode instanceof Join) { + final BeamJoin join = (BeamJoin) relNode; + schemaMaps.put(RexBeamUtils.getSchemaName(((BeamNode) join.getLeft()).getVariableName()), + join.getLeft().getRowType()); + schemaMaps.put(RexBeamUtils.getSchemaName(((BeamNode) join.getRight()).getVariableName()), + join.getRight().getRowType()); + } + } + } + + protected void relNodeToJava(RelNode relNode, List statements) { + final BeamNode beamNode = (BeamNode) relNode; + final String nodeName = beamNode.getVariableName(); + if (visitedNodes.contains(nodeName)) { + return; + } + + for (RelNode relNode1 : relNode.getInputs()) { + relNodeToJava(relNode1, statements); + } + beamNodeToStatement(beamNode, statements); + visitedNodes.add(nodeName); + } + + /** + * Generates statements to initialize required static variables. + */ + Statement initStaticVars(List memberDeclarations) { + final List ifBodyStatements = new ArrayList<>(); + final Set resourceFilesExprSet = new LinkedHashSet<>(); + final List udfStatements = initPigUDFs(memberDeclarations, resourceFilesExprSet); + + // Then add statements to initialize Pig UDFs + ifBodyStatements.addAll(udfStatements); + + // Wrap them with a guard to make sure they get initialized just once + if (!ifBodyStatements.isEmpty()) { + final ParameterExpression udfStaticInitCopyResourceDone = Expressions.parameter(boolean.class, "staticVarInit"); + memberDeclarations.add(0, + Expressions.fieldDecl(Modifier.PRIVATE | Modifier.STATIC, udfStaticInitCopyResourceDone, + Expressions.constant(false, boolean.class))); + ifBodyStatements.add(Expressions.statement( + Expressions.assign(udfStaticInitCopyResourceDone, Expressions.constant(true, boolean.class)))); + return Expressions.ifThen(Expressions.not(udfStaticInitCopyResourceDone), Expressions.block(ifBodyStatements)); + } + return null; + } + + /** + * Initializes Pig UDF and capture resource files needed to initialize these UDF objects. + * @param memberDeclarations Output param for UDF object declaration + * @param resourceFilesExprSet Output param for resource files needed + */ + private List initPigUDFs(List memberDeclarations, Set resourceFilesExprSet) { + final List udfInits = new ArrayList<>(); + final Set exceptions = new HashSet<>(); + + for (PigUserDefinedFunction pigUDF : pigUdfMap.values()) { + final Class clazz = ((ScalarFunctionImpl) pigUDF.getFunction()).method.getDeclaringClass(); + if (EXCLUDED_PIG_UDFS.contains(clazz)) { + continue; + } + + List constructorArgs = new ArrayList<>(); + if (pigUDF.funcSpec.getCtorArgs() != null) { + final String[] ctorArgs = pigUDF.funcSpec.getCtorArgs(); + final Class[] ctorTypes = new Class[ctorArgs.length]; + for (int i = 0; i < ctorArgs.length; i++) { + final String filename = ctorArgs[i].substring(ctorArgs[i].lastIndexOf("/") + 1); + constructorArgs.add(Expressions.constant(filename, String.class)); + ctorTypes[i] = String.class; + } + try { + Collections.addAll(exceptions, clazz.getConstructor(ctorTypes).getExceptionTypes()); + } catch (NoSuchMethodException e) { + e.printStackTrace(); + StringBuilder message = new StringBuilder("Cannot find constructor for Pig UDF ").append(clazz.getName()) + .append(" with arguments: ") + .append(Arrays.asList(ctorTypes)); + throw new RuntimeException(message.toString()); + } + if (clazz == JythonFunction.class) { + // If Python UDF, only need to copy the python source file, which is the first argument + resourceFilesExprSet.add(constructorArgs.get(0)); + } else { + for (ConstantExpression arg : constructorArgs) { + resourceFilesExprSet.add(arg); + } + } + } + final ParameterExpression udfName = PigUDFUtils.getPigUDFParamExpr(pigUDF); + memberDeclarations.add(Expressions.fieldDecl(Modifier.PRIVATE | Modifier.STATIC, udfName)); + udfInits.add(Expressions.statement(Expressions.assign(udfName, Expressions.new_(clazz, constructorArgs)))); + } + + if (exceptions.isEmpty()) { + return udfInits; + } + return RexBeamUtils.wrapTryCatch(udfInits, exceptions).statements; + } + + /** + * + * @param beamPlan Beam physical plan + * @return Java source code string for the Beam application + */ + protected abstract String toJavaCode(RelNode beamPlan); + + protected abstract void beamNodeToStatement(BeamNode beamNode, List statements); + + private String getRecordName(RelDataType recordType) { + if (recordType instanceof DynamicRecordType) { + return BeamExecUtil.DYNAMIC_RECORD_NAME_PREFIXED + (recordIndex++); + } + return BeamExecUtil.RECORD_NAME_PREFIXED; + } + + private FieldDeclaration makeSchemaDeclaration(ParameterExpression schemaVar, RelDataType rowType) { + final String schemaString = new RelDataTypeToAvro().convertRelDataType(rowType, getRecordName(rowType)).toString(); + return Expressions.fieldDecl(Modifier.PRIVATE | Modifier.FINAL | Modifier.STATIC, schemaVar, + Expressions.call(Expressions.new_(Schema.Parser.class), PARSE, Expressions.constant(schemaString))); + } + + // HACKED: Fixing two problems generated by Calcite code optimization: + // 1. Declare static variable in anonymous class (in lambda functions) -> remove static + // 2. Have invalid character '$' in the variable name -> replace '$' by '_' + private static final Pattern STATIC_VAR_PATTERN = Pattern.compile("" + + "(?im)" // case-insensitive, multi-line + + "\\s*(static\\s+final\\s+" // [static final ]Type varName + + "[^\\s]+\\s+" // static final [Type ]varName + + "([^\\s]*))"); // static final Type [varName] ($1) + private static String fixVarName(String code) { + String newCode = code; + final Matcher matcher = STATIC_VAR_PATTERN.matcher(newCode); + while (matcher.find()) { + String match1 = matcher.group(1); + String match2 = matcher.group(2); + if (match2.contains("$")) { + // Cannot declare static var in an anonymous class, remove it. + final String noStatic = match1.replace("static", ""); + newCode = newCode.replace(match1, noStatic); + // Replace $ by _ + String varName = match2.replace("$", "_"); + newCode = newCode.replace(match2, varName); + } + } + return newCode; + } + + private static final Pattern PATH_PATTERN = Pattern.compile("(\\w+(\\.\\w+)+)"); + private static String extractPathForImport(String source) { + Map pathCount = new HashMap<>(); + final Matcher matcher = PATH_PATTERN.matcher(source); + while (matcher.find()) { + final String match = matcher.group(1); + if (match != null) { + pathCount.put(match, pathCount.getOrDefault(match, 0) + 1); + } + } + final Map replacements = new HashMap<>(); + pathCount.forEach((k, v) -> { + String path = k; + try { + Class.forName(path); + } catch (ClassNotFoundException e) { + path = path.substring(0, path.lastIndexOf(".")); + try { + Class.forName(path); + } catch (ClassNotFoundException e1) { + int dotIndex = path.lastIndexOf("."); + if (dotIndex < 0) { + return; + } + path = path.substring(0, path.lastIndexOf(".")); + try { + Class.forName(path); + } catch (ClassNotFoundException e2) { + return; + } + } + } + final String className = path.substring(path.lastIndexOf('.') + 1); + final String replacementName = replacements.get(className); + final Integer replacementCount = pathCount.get(replacementName); + if (replacementName == null || (replacementCount != null && replacementCount < v)) { + replacements.put(className, path); + } + }); + final StringBuilder stringBuilder = new StringBuilder(); + replacements.values().stream().sorted().forEach(v -> stringBuilder.append("import ").append(v).append(";\n")); + String newSource = source; + for (String key : replacements.keySet()) { + newSource = newSource.replace(replacements.get(key), key); + } + + return stringBuilder.append("\n").toString() + newSource; + } + + /** + * Gets Calcite plan for a Pig script. + * + * @param pigFile Path to Pig script + * @param params Pig params + */ + private static RelNode getRelNodeFromPigFile(String pigFile, Map params) throws Exception { + PigConverter pigConverter = PigConverter.create(Frameworks.newConfigBuilder().build()); + pigConverter.setValidateEachStatement(true); + List relNodes = pigConverter.pigScript2Rel(pigFile, params, false); + assert relNodes.size() == 1 : "Expect only one store statement in the Pig script"; + return relNodes.get(0); + } + + /** + * Gets Calcite plan for a Pig script. + * + * @param pigQuery Pig query string + */ + private static RelNode getRelNodeFromPigQuery(String pigQuery) throws Exception { + PigConverter pigConverter = PigConverter.create(Frameworks.newConfigBuilder().build()); + pigConverter.setValidateEachStatement(true); + List relNodes = pigConverter.pigQuery2Rel(pigQuery, false, true, false); + assert relNodes.size() == 1 : "Expect only one store statement in the Pig script"; + return relNodes.get(0); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamAggregateRule.java b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamAggregateRule.java new file mode 100644 index 000000000..cb751b571 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamAggregateRule.java @@ -0,0 +1,34 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner.rules; + +import com.linkedin.beam.operators.BeamAggregate; +import com.linkedin.beam.planner.CalciteBeamConfig; +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.planner.BeamPlanner; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalAggregate; + + +public class BeamAggregateRule extends ConverterRule { + public BeamAggregateRule() { + super(LogicalAggregate.class, Convention.NONE, BeamConvention.INSTANCE, "BeamAggregateRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final LogicalAggregate aggregate = (LogicalAggregate) rel; + final CalciteBeamConfig calciteBeamConfig = ((BeamPlanner) rel.getCluster().getPlanner())._calciteBeamConfig; + return BeamAggregate.create( + convert(aggregate.getInput(), aggregate.getInput().getTraitSet().replace(BeamConvention.INSTANCE)), + aggregate.getGroupSet(), + aggregate.getGroupSets(), + aggregate.getAggCallList(), + calciteBeamConfig.aggregateWindowMinutes); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamArrayFlattenRule.java b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamArrayFlattenRule.java new file mode 100644 index 000000000..bf5f2c424 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamArrayFlattenRule.java @@ -0,0 +1,116 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner.rules; + +import com.linkedin.beam.operators.BeamArrayFlatten; +import com.linkedin.beam.planner.BeamConvention; +import java.util.ArrayList; +import java.util.List; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.volcano.RelSubset; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.core.Correlate; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.core.Uncollect; +import org.apache.calcite.rel.logical.LogicalCorrelate; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexFieldAccess; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; + + +public class BeamArrayFlattenRule extends ConverterRule { + + public BeamArrayFlattenRule() { + super(LogicalCorrelate.class, Convention.NONE, BeamConvention.INSTANCE, "BeamArrayFlattenRule"); + } + + /** + * That rule only applies for Pig DataBag flatten, which is translated as correlate operator + * in Calcite logical plan. + */ + @Override + public RelNode convert(RelNode rel) { + final RelDataTypeFactory typeFactory = rel.getCluster().getTypeFactory(); + final LogicalCorrelate correlate = (LogicalCorrelate) rel; + final RelNode left = getLeft(correlate); + if (!((left instanceof Project || left instanceof TableScan) && getRight(correlate) instanceof Uncollect)) { + unsupported(); + } + + final RelDataType leftType = getLeft(correlate).getRowType(); + final List resultTypes = new ArrayList<>(); + final List resultNames = new ArrayList<>(); + for (RelDataTypeField field : leftType.getFieldList()) { + resultTypes.add(field.getType()); + resultNames.add(field.getName()); + } + + final Uncollect right = (Uncollect) getRight(correlate); + if (!(getInput(right) instanceof Project)) { + unsupported(); + } + final Project rightInput = (Project) getInput(right); + final List flattenCols = new ArrayList<>(); + for (RexNode rex : rightInput.getChildExps()) { + if (rex.getKind() != SqlKind.FIELD_ACCESS) { + unsupported(); + } + final RexFieldAccess fieldAccess = (RexFieldAccess) rex; + final RelDataTypeField leftField = leftType.getField(fieldAccess.getField().getName(), true, false); + if (fieldAccess.getReferenceExpr().getKind() != SqlKind.CORREL_VARIABLE + || leftField == null + || leftField.getType().getComponentType() == null) { + unsupported(); + } + flattenCols.add(leftField.getName()); + final RelDataType componentType = leftField.getType().getComponentType(); + if (componentType.isStruct()) { + for (RelDataTypeField field : componentType.getFieldList()) { + resultTypes.add(field.getType()); + resultNames.add(field.getName()); + } + } else { + resultTypes.add(componentType); + resultNames.add(correlate.getRowType().getFieldNames().get(resultTypes.size() - 1)); + } + } + + return BeamArrayFlatten.create( + convert(correlate.getLeft(), correlate.getLeft().getTraitSet().replace(BeamConvention.INSTANCE)), + flattenCols, + typeFactory.createStructType(resultTypes, resultNames)); + } + + private RelNode getLeft(Correlate correlate) { + if (correlate.getLeft() instanceof RelSubset) { + return ((RelSubset) correlate.getLeft()).getOriginal(); + } + return correlate.getLeft(); + } + + private RelNode getRight(Correlate correlate) { + if (correlate.getRight() instanceof RelSubset) { + return ((RelSubset) correlate.getRight()).getOriginal(); + } + return correlate.getRight(); + } + + private RelNode getInput(Uncollect uncollect) { + if (uncollect.getInput() instanceof RelSubset) { + return ((RelSubset) uncollect.getInput()).getOriginal(); + } + return uncollect.getInput(); + } + + private void unsupported() { + throw new UnsupportedOperationException("Not supporting correlates other than Array Flatten"); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamFilterRule.java b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamFilterRule.java new file mode 100644 index 000000000..0052175cf --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamFilterRule.java @@ -0,0 +1,31 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner.rules; + +import com.linkedin.beam.operators.BeamFilter; +import com.linkedin.beam.planner.BeamConvention; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalFilter; + + +public class BeamFilterRule extends ConverterRule { + public BeamFilterRule() { + super(LogicalFilter.class, RelOptUtil.FILTER_PREDICATE, Convention.NONE, + BeamConvention.INSTANCE, "BeamFilterRule"); + } + @Override + public RelNode convert(RelNode rel) { + final LogicalFilter filter = (LogicalFilter) rel; + return BeamFilter.create( + convert(filter.getInput(), + filter.getInput().getTraitSet() + .replace(BeamConvention.INSTANCE)), + filter.getCondition()); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamJoinRule.java b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamJoinRule.java new file mode 100644 index 000000000..75387b90c --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamJoinRule.java @@ -0,0 +1,45 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner.rules; + +import com.linkedin.beam.operators.BeamJoin; +import com.linkedin.beam.planner.BeamConvention; +import com.linkedin.beam.planner.BeamPlanner; +import com.linkedin.beam.planner.CalciteBeamConfig; +import java.util.ArrayList; +import java.util.List; +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.JoinInfo; +import org.apache.calcite.rel.logical.LogicalJoin; + + +public class BeamJoinRule extends ConverterRule { + public BeamJoinRule() { + super(LogicalJoin.class, Convention.NONE, BeamConvention.INSTANCE, "BeamJoinRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final LogicalJoin join = (LogicalJoin) rel; + final List newInputs = new ArrayList<>(); + for (RelNode input : join.getInputs()) { + newInputs.add(convert(input, input.getTraitSet().replace(BeamConvention.INSTANCE))); + } + RelNode left = newInputs.get(0); + RelNode right = newInputs.get(1); + final JoinInfo info = JoinInfo.of(left, right, join.getCondition()); + if (!info.isEqui()) { + // Only accept equi-join + throw new UnsupportedOperationException("None equi-join not supported"); + } + + final CalciteBeamConfig calciteBeamConfig = ((BeamPlanner) rel.getCluster().getPlanner())._calciteBeamConfig; + return BeamJoin.create(left, calciteBeamConfig.timestampField, right, calciteBeamConfig.timestampField, + join.getCondition(), join.getVariablesSet(), join.getJoinType(), calciteBeamConfig.joinWindowMinutes); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamProjectRule.java b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamProjectRule.java new file mode 100644 index 000000000..882cca4bd --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamProjectRule.java @@ -0,0 +1,32 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner.rules; + +import com.linkedin.beam.operators.BeamProject; +import com.linkedin.beam.planner.BeamConvention; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalProject; + +public class BeamProjectRule extends ConverterRule { + public BeamProjectRule() { + super(LogicalProject.class, RelOptUtil.PROJECT_PREDICATE, Convention.NONE, + BeamConvention.INSTANCE, "ScanProjectRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final LogicalProject project = (LogicalProject) rel; + return BeamProject.create( + convert(project.getInput(), + project.getInput().getTraitSet() + .replace(BeamConvention.INSTANCE)), + project.getProjects(), + project.getRowType()); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamTableScanRule.java b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamTableScanRule.java new file mode 100644 index 000000000..c0241259d --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamTableScanRule.java @@ -0,0 +1,27 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner.rules; + +import com.linkedin.beam.operators.BeamTableScan; +import com.linkedin.beam.planner.BeamConvention; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalTableScan; + + +public class BeamTableScanRule extends ConverterRule { + public BeamTableScanRule() { + super(LogicalTableScan.class, Convention.NONE, + BeamConvention.INSTANCE, "BeamTableScanRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final LogicalTableScan scan = (LogicalTableScan) rel; + return BeamTableScan.create(scan.getCluster(), scan.getTable()); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamUnionRule.java b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamUnionRule.java new file mode 100644 index 000000000..31234dee6 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/planner/rules/BeamUnionRule.java @@ -0,0 +1,27 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner.rules; + +import com.linkedin.beam.operators.BeamUnion; +import com.linkedin.beam.planner.BeamConvention; +import org.apache.calcite.plan.Convention; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.convert.ConverterRule; +import org.apache.calcite.rel.logical.LogicalUnion; + + +public class BeamUnionRule extends ConverterRule { + public BeamUnionRule() { + super(LogicalUnion.class, Convention.NONE, BeamConvention.INSTANCE, + "BeamUnionRule"); + } + + @Override + public RelNode convert(RelNode rel) { + final LogicalUnion union = (LogicalUnion) rel; + return BeamUnion.create(convertList(union.getInputs(), BeamConvention.INSTANCE), union.all); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/utils/AvroJavaTypeFactory.java b/coral-beam/src/main/java/com/linkedin/beam/utils/AvroJavaTypeFactory.java new file mode 100644 index 000000000..c1268c760 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/utils/AvroJavaTypeFactory.java @@ -0,0 +1,67 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.utils; + +import java.lang.reflect.Type; +import java.nio.ByteBuffer; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericRecord; +import org.apache.calcite.jdbc.JavaTypeFactoryImpl; +import org.apache.calcite.linq4j.tree.Primitive; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.type.SqlTypeName; + + +public class AvroJavaTypeFactory extends JavaTypeFactoryImpl { + public static final AvroJavaTypeFactory AVRO_TYPE_FACTORY = new AvroJavaTypeFactory(); + + @Override + public Type getJavaClass(RelDataType type) { + SqlTypeName typeName = type.getSqlTypeName(); + switch (typeName) { + case ROW: + return GenericRecord.class; + case BINARY: + return ByteBuffer.class; + case MULTISET: + return GenericArray.class; + default: + return super.getJavaClass(type); + } + } + + /** + * Gets Java mutable type (primitive type if applicable) for a rel data type + * + * @param type Rel data type + * @return Java mutable type + */ + public Type getMutableJavaClass(RelDataType type) { + final Type originalType = getJavaClass(type); + final Primitive primitive = Primitive.ofBox(originalType); + if (primitive != null) { + return primitive.primitiveClass; + } + return originalType; + } + + /** + * Gets Java immutable type for a rel data type. If corresponding Java type is primitive, return the box type. + * (like Integer instead of int). + * + * @param type Rel data type + * @return Java immutable type + */ + public Type getImmutableJavaClass(RelDataType type) { + final Type originalType = getJavaClass(type); + final Primitive primitive = Primitive.of(originalType); + if (primitive != null) { + return primitive.boxClass; + } + return originalType; + } + +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/utils/MethodNames.java b/coral-beam/src/main/java/com/linkedin/beam/utils/MethodNames.java new file mode 100644 index 000000000..4fba3d6ad --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/utils/MethodNames.java @@ -0,0 +1,85 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.utils; + +public class MethodNames { + private MethodNames() { + } + + // Beam methods + public static final String APPLY = "apply"; + public static final String INTO = "into"; + public static final String VIA = "via"; + public static final String BY = "by"; + public static final String AND = "and"; + public static final String WITH = "with"; + public static final String WITH_IN = "within"; + public static final String WITH_NAME = "withName"; + public static final String WITH_TOPIC = "withTopic"; + public static final String WITH_KEY_TYPE = "withKeyType"; + public static final String WITH_TIME_FUNCTION = "withTimeFunction"; + public static final String WITH_TIMESTAMP_FN = "withTimestampFn"; + public static final String WITH_STORE_TYPE = "withStoreType"; + public static final String WITHOUT_REPARTITION = "withoutRepartition"; + public static final String WITHOUT_METADATA = "withoutMetadata"; + public static final String PER_KEY = "perKey"; + public static final String STRINGS = "strings"; + public static final String CREATE = "create"; + public static final String RUN = "run"; + public static final String P_COLLECTIONS = "pCollections"; + + // Time methods + public static final String STANDARD_DAYS = "standardDays"; + public static final String STANDARD_MINUTES = "standardMinutes"; + + // BeamExecUtil + public static final String RAND = "rand"; + public static final String READ = "read"; + public static final String WRITE = "write"; + public static final String TO_AVRO_ARRAY = "toAvroArray"; + public static final String TO_AVRO_RECORD = "toAvroRecord"; + public static final String STRING_GENERIC_RECORD_WRITE = "stringGenericRecordWrite"; + public static final String TO_BYTE_BUFFER = "toByteBuffer"; + public static final String BUILD_STRING_KEY = "buildStringKey"; + public static final String ROUND_UP_TIME = "roundUpTime"; + public static final String BUILD_DISTINCT_STRING_KEY_FROM_RECORD = "buildDistinctStringKeyFromRecord"; + + // GenericRecord + public static final String AVRO_PUT = "put"; + public static final String AVRO_GET = "get"; + public static final String GET_KEY = "getKey"; + public static final String GET_VALUE = "getValue"; + + // PigUDF + public static final String EXEC = "exec"; + public static final String BUILD_TUPLE = "buildTuple"; + + // String + public static final String EQUALS = "equals"; + public static final String TO_LOWER = "toLowerCase"; + public static final String TO_UPPER = "toUpperCase"; + public static final String COMPARE = "compare"; + + // Map + public static final String MAP_GET = "get"; + + + // Iterator.class + public static final String NEXT = "next"; + public static final String HAS_NEXT = "hasNext"; + + // Other classes + public static final String OF = "of"; // ImmutableList.class + public static final String MAIN = "main"; // main method + public static final String FLATTEN = "flatten"; // ArrayFlatten.class + public static final String ITERATOR = "iterator"; // Collection.class + + public static final String PARSE = "parse"; // Schema.Parser.class + public static final String VALUE_OF = "valueOf"; + public static final String MATCHES = "matches"; // Pattern + public static final String ARRAY = "array"; // DataByteArray + public static final String PRINT_STACK_TRACE = "printStackTrace"; //Exception +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/utils/Methods.java b/coral-beam/src/main/java/com/linkedin/beam/utils/Methods.java new file mode 100644 index 000000000..8cde55bbc --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/utils/Methods.java @@ -0,0 +1,31 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.utils; + +import java.lang.reflect.Method; +import java.util.Objects; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.piglet.PigRelSqlUdfs; + + +public class Methods { + private Methods() { + } + + public static final Method P_COLLECTION_SET_CODER = Types.lookupMethod(PCollection.class, "setCoder", Coder.class); + public static final Method P_COLLECTION_APPLY = Types.lookupMethod(PCollection.class, "apply", PTransform.class); + public static final Method AVRO_PUT = Types.lookupMethod(GenericRecord.class, "put", String.class, Object.class); + public static final Method OBJECTS_TO_STRING = Types.lookupMethod(Objects.class, "toString", Object.class, String.class); + public static final Method BUILD_TUPLE = Types.lookupMethod(PigRelSqlUdfs.class, "buildTuple", Object.class); + public static final Method BUILD_BAG = Types.lookupMethod(PigRelSqlUdfs.class, "buildBag", Object.class); + public static final Method AVRO_ARRAY_ADD = Types.lookupMethod(GenericArray.class, "add", Object.class); + +} \ No newline at end of file diff --git a/coral-beam/src/main/java/com/linkedin/beam/utils/PigUDFUtils.java b/coral-beam/src/main/java/com/linkedin/beam/utils/PigUDFUtils.java new file mode 100644 index 000000000..255ff2f13 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/utils/PigUDFUtils.java @@ -0,0 +1,116 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.utils; + +import java.lang.reflect.Method; +import java.util.Map; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.piglet.PigUserDefinedFunction; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelVisitor; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.schema.impl.ScalarFunctionImpl; +import org.apache.pig.scripting.jython.JythonFunction; + + +public class PigUDFUtils { + + private PigUDFUtils() { + } + + /** + * Collects all Pig UDF calls in a given Calcite logical plan. + * + * @param rel Root of the Calcite logical plan + * @param udfMap Map to store all Pig UDF references in the plan + */ + public static void collectPigUDFs(RelNode rel, Map udfMap) { + new RelUDFCollector(udfMap).go(rel); + } + + + /** + * Gets the key in UDF map for a given Pig UDF. The key will be unique for each UDF. + * + * @param pigUDF Pig UDF + * @return a string key to represent the Pig UDF + */ + public static String getUDFMapKey(PigUserDefinedFunction pigUDF) { + final Method udfMethod = ((ScalarFunctionImpl) pigUDF.getFunction()).method; + final Class clazz = udfMethod.getDeclaringClass(); + final String className = clazz.getName().replace(".", "_"); + if (clazz == JythonFunction.class) { + final String[] args = pigUDF.funcSpec.getCtorArgs(); + assert args != null && args.length == 2; + final String fileName = args[0].substring(args[0].lastIndexOf("/") + 1, args[0].lastIndexOf(".py")); + // key = [clas name]_[file name]_[function name] + return className + "_" + fileName + "_" + args[1]; + } + return className; + } + + /** + * Gets the variable represent PIG UDF object in autogen code. + * + * @param pigUDF Pig UDF. + * @return A {@link ParameterExpression} to represent the Pig UDF object + */ + public static ParameterExpression getPigUDFParamExpr(PigUserDefinedFunction pigUDF) { + final Class clazz = ((ScalarFunctionImpl) pigUDF.getFunction()).method.getDeclaringClass(); + return Expressions.parameter(clazz, getUDFMapKey(pigUDF)); + } + + /** + * Rel visitor to collect all Pig UDF calls in a relational plan. + */ + static class RelUDFCollector extends RelVisitor { + final Map udfMap; + RelUDFCollector(Map udfMap) { + this.udfMap = udfMap; + } + + @Override + public void visit(RelNode node, int ordinal, RelNode parent) { + if (node instanceof Project) { + for (RexNode rexNode : ((Project) node).getProjects()) { + rexNode.accept(new RexUDFCollector(udfMap)); + } + } + if (node instanceof Filter) { + ((Filter) node).getCondition().accept(new RexUDFCollector(udfMap)); + } + super.visit(node, ordinal, parent); + } + } + + /** + * Rex visitor to collect all Pig UDF calls in an relational expression. + */ + private static class RexUDFCollector extends RexVisitorImpl { + final Map udfMap; + + RexUDFCollector(Map udfMap) { + super(true); + this.udfMap = udfMap; + } + + @Override + public Void visitCall(RexCall call) { + if (call.getOperator() instanceof PigUserDefinedFunction) { + PigUserDefinedFunction pigFunc = (PigUserDefinedFunction) call.getOperator(); + if (pigFunc.funcSpec != null) { + udfMap.put(getUDFMapKey(pigFunc), pigFunc); + } + } + return super.visitCall(call); + } + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/utils/RelDataTypeToAvro.java b/coral-beam/src/main/java/com/linkedin/beam/utils/RelDataTypeToAvro.java new file mode 100644 index 000000000..754161d2d --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/utils/RelDataTypeToAvro.java @@ -0,0 +1,120 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.utils; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.calcite.rel.type.DynamicRecordType; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rel.type.RelRecordType; +import org.apache.calcite.sql.type.ArraySqlType; +import org.apache.calcite.sql.type.BasicSqlType; +import org.apache.calcite.sql.type.MapSqlType; +import org.apache.calcite.sql.type.MultisetSqlType; +import org.apache.calcite.sql.type.SqlTypeName; + + +public class RelDataTypeToAvro { + public static final String DEFAULT_RECORD_NAME = "record"; + + private int recordId; + + public RelDataTypeToAvro() { + recordId = 0; + } + + public Schema convertRelDataType(RelDataType relType) { + return convertRelDataType(relType, null); + } + + public Schema convertRelDataType(RelDataType relType, String recName) { + final Schema avroSchema = convertRelDataTypeNoneNull(relType, recName); + if (relType.isNullable() && avroSchema.getType() != Schema.Type.NULL) { + return Schema.createUnion(Arrays.asList(avroSchema, Schema.create(Schema.Type.NULL))); + } + return avroSchema; + } + + public Schema convertRelDataTypeNoneNull(RelDataType relType, String recName) { + if (relType instanceof RelRecordType || relType instanceof DynamicRecordType) { + return convertRecord(relType, recName); + } + if (relType instanceof BasicSqlType) { + return convertBasicSqlType((BasicSqlType) relType); + } + if (relType instanceof MultisetSqlType || relType instanceof ArraySqlType) { + return Schema.createArray(convertRelDataType(relType.getComponentType(), recName)); + } + if (relType instanceof MapSqlType) { + final MapSqlType mapSqlType = (MapSqlType) relType; + if (!SqlTypeName.CHAR_TYPES.contains(mapSqlType.getKeyType().getSqlTypeName())) { + throw new UnsupportedOperationException("Key of Map can only be a String: " + mapSqlType.getKeyType().getSqlTypeName().getName()); + } + return Schema.createMap(convertRelDataType(mapSqlType.getValueType(), recName)); + } + + throw new UnsupportedOperationException("Unsupported RelDataType from class: " + relType.getClass().toString()); + } + + private Schema convertBasicSqlType(BasicSqlType relType) { + switch (relType.getSqlTypeName()) { + case BOOLEAN: + return Schema.create(Schema.Type.BOOLEAN); + case TINYINT: + case INTEGER: + return Schema.create(Schema.Type.INT); + case BIGINT: + return Schema.create(Schema.Type.LONG); + case FLOAT: + return Schema.create(Schema.Type.FLOAT); + case DOUBLE: + return Schema.create(Schema.Type.DOUBLE); + case VARCHAR: + case CHAR: + return Schema.create(Schema.Type.STRING); + case BINARY: + return Schema.create(Schema.Type.BYTES); + case NULL: + return Schema.create(Schema.Type.NULL); + case ANY: + return Schema.create(Schema.Type.BYTES); + default: + throw new UnsupportedOperationException(relType.getSqlTypeName() + " is not supported."); + } + } + + public Schema convertRecord(RelDataType relRecord) { + return convertRecord(relRecord, null); + } + + public Schema convertRecord(RelDataType relRecord, String name) { + return convertRecord(relRecord, null, name, "rel_avro", null); + } + + public Schema convertRecord(RelDataType relRecord, List columnComments, String name, String namespace, + String doc) { + final List fields = new ArrayList(); + String schemaName = name != null && !name.isEmpty() ? name : DEFAULT_RECORD_NAME; + for (RelDataTypeField relField : relRecord.getFieldList()) { + final String comment = columnComments != null && columnComments.size() > relField.getIndex() ? columnComments.get( + relField.getIndex()) : null; + final String fieldSchemaName = schemaName + "_" + toAvroQualifedName(relField.getName()); + fields.add(new Field(toAvroQualifedName(relField.getName()), convertRelDataType(relField.getType(), fieldSchemaName), comment, null)); + } + + final Schema avroSchema = Schema.createRecord(schemaName, doc, namespace, false); + avroSchema.setFields(fields); + return avroSchema; + } + + public static String toAvroQualifedName(String relName) { + return relName.replace("$", "_"); + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/utils/RexBeamUtils.java b/coral-beam/src/main/java/com/linkedin/beam/utils/RexBeamUtils.java new file mode 100644 index 000000000..eaeefc227 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/utils/RexBeamUtils.java @@ -0,0 +1,358 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.utils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; +import com.linkedin.beam.excution.BeamExecUtil; +import java.lang.reflect.Modifier; +import java.lang.reflect.Type; +import java.lang.reflect.TypeVariable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.util.Utf8; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.calcite.linq4j.tree.BlockStatement; +import org.apache.calcite.linq4j.tree.Blocks; +import org.apache.calcite.linq4j.tree.CatchBlock; +import org.apache.calcite.linq4j.tree.ConstantExpression; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.ExpressionType; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.MemberDeclaration; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Primitive; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.linq4j.tree.UnaryExpression; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.commons.lang.ObjectUtils; + +import static com.linkedin.beam.utils.MethodNames.*; + + +public class RexBeamUtils { + private static final Set SUPPORTED_STRING_CONVERSION_CLASSES = Sets.newHashSet( + String.class, + int.class, + Integer.class, + double.class, + Double.class, + long.class, + Long.class, + boolean.class, + Boolean.class, + Utf8.class, + Map.class, + ByteBuffer.class, + GenericRecord.class, + Object.class); + + + private RexBeamUtils() { + } + + public static Expression makeLambdaFunction(Type functionType, Type returnType, List arguments, + List bodyStatements, Set exceptions) { + return Expressions.new_( + functionType, + Collections.emptyList(), + Arrays.asList( + Expressions.methodDecl( + Modifier.PUBLIC, + returnType, + MethodNames.APPLY, + arguments, + wrapTryCatch(bodyStatements, exceptions)))); + } + + public static BlockStatement wrapTryCatch(List bodyStatements, Set exceptions) { + final BlockStatement body = Expressions.block(bodyStatements); + if (exceptions != null && !exceptions.isEmpty()) { + final List catches = new ArrayList<>(); + int id = 1; + final List sortedExceptions = sortException(exceptions); + for (Type exceptionType : sortedExceptions) { + final ParameterExpression exceptionName = Expressions.parameter(exceptionType, "e" + (id++)); + final List catchStatements = new ArrayList<>(); + catchStatements.add(Expressions.statement(Expressions.call(exceptionName, MethodNames.PRINT_STACK_TRACE))); + final String message = "Encounter " + exceptionType.getTypeName() + " during execution. Please check the stack trace"; + catchStatements.add(Expressions.throw_(Expressions.new_(Error.class, + Expressions.constant(message, String.class), exceptionName))); + catches.add(Expressions.catch_(exceptionName, Expressions.block(catchStatements))); + } + return Expressions.block(Expressions.tryCatch(body, catches)); + } + return body; + } + + private static List sortException(Set origins) { + List sortedList = new ArrayList<>(origins); + sortedList.sort((o1, o2) -> { + if (o1.equals(o2)) { + return 0; + } + if (o1.isAssignableFrom(o2)) { + return 1; + } + + if (o2.isAssignableFrom(o1)) { + return -1; + } + return o1.getName().compareTo(o2.getName()); + }); + return sortedList; + } + + public static Expression stripCasting(Expression expr) { + Expression targetExpr = expr; + // Remove any casting before to get the original expression + while (targetExpr.getNodeType() == ExpressionType.Convert) { + targetExpr = ((UnaryExpression) targetExpr).expression; + } + return targetExpr; + } + + public static Expression toObjectType(Expression expr) { + Expression targetExpr = stripCasting(expr); + if (expr instanceof ConstantExpression && Primitive.is(expr.getType())) { + // Object cast for negative numeric like (Object) -1 does not compile in Java + // HACK: convert the expression to (Object) Integer.valueOf(-1) + final ConstantExpression constant = (ConstantExpression) expr; + if (expr.getType() == int.class) { + if ((int) constant.value < 0) { + targetExpr = Expressions.call(Integer.class, VALUE_OF, constant); + } + } else if (expr.getType() == long.class) { + if ((long) constant.value < 0) { + targetExpr = Expressions.call(Long.class, VALUE_OF, constant); + } + } else if (expr.getType() == double.class) { + if ((double) constant.value < 0) { + targetExpr = Expressions.call(Double.class, VALUE_OF, constant); + } + } + } + + if (targetExpr.getType() != Object.class) { + return Expressions.convert_(targetExpr, Object.class); + } + return targetExpr; + } + + public static Expression toJavaStringType(Expression expr) { + if (SUPPORTED_STRING_CONVERSION_CLASSES.contains(expr.getType())) { + if (expr.getType() == String.class) { + return expr; + } + if (expr.getType() == int.class + || expr.getType() == double.class + || expr.getType() == long.class + || expr.getType() == boolean.class) { + return Expressions.call(String.class, MethodNames.VALUE_OF, expr); + } else { + return Expressions.call(Methods.OBJECTS_TO_STRING, ImmutableList.of(expr, Expressions.constant(null))); + } + } + // Allow value of any type in a map to be converted into a String + if (expr.getType() instanceof TypeVariable + && ((TypeVariable) expr.getType()).getGenericDeclaration() == Map.class + && ((TypeVariable) expr.getType()).getName().equals("V")) { + return Expressions.call(Methods.OBJECTS_TO_STRING, ImmutableList.of(expr, Expressions.constant(null))); + } + throw new UnsupportedOperationException("Type " + expr.getType() + " conversion to JavaStringType not supported "); + } + + public static Expression stringCompareExpr(Expression op1, Expression op2) { + return Expressions.call(ObjectUtils.class, MethodNames.COMPARE, op1, op2); + } + + public static String getSchemaName(String varName) { + return varName.toUpperCase() + "_SCHEMA"; + } + + public static ParameterExpression getSchemaParam(String paramName) { + return Expressions.parameter(Schema.class, paramName); + } + + /** + * Builds an exprssion to check whether any expression in an expression list is null. + * + * @param exprs List of expression + * @return Condition expression for checking if any of the expressions is null. + */ + static Expression buildNullSafeExpr(List exprs) { + Expression result = notNull(exprs.get(0)); + final Set visited = new HashSet<>(); + visited.add(exprs.get(0)); + for (int i = 1; i < exprs.size(); i++) { + if (exprs.get(i).getNodeType() != ExpressionType.Constant + && !visited.contains(exprs.get(i))) { + result = Expressions.andAlso(result, notNull(exprs.get(i))); + visited.add(exprs.get(i)); + } + } + return result; + } + + /** + * Checks if an expression is not null. + * + * @return Condition expression for the check + */ + static Expression notNull(Expression expr) { + if (Primitive.is(expr.getType())) { + return Expressions.constant(true); + } + return Expressions.notEqual(expr, Expressions.constant(null)); + } + + /** + * Casts an expression to a type specified by a rel data type. + * + * @param expr Expression to cast + * @param targetType The target rel data type + * @return Cast expression + */ + public static Expression convertType(Expression expr, RelDataType targetType) { + // Target type should be nullable (or immutable) to be generic. + final Type targetJavaType = AvroJavaTypeFactory.AVRO_TYPE_FACTORY.getImmutableJavaClass(targetType); + final Type exprType = expr.getType(); + + if (exprType == targetJavaType || exprType == Types.unbox(targetJavaType)) { + // Same type, no cast needed + return expr; + } + + if (targetType.getSqlTypeName().getFamily() == SqlTypeFamily.CHARACTER) { + // Convert to String type + return toJavaStringType(expr); + } + + if (targetType.getSqlTypeName() == SqlTypeName.BINARY) { + // Convert to binary type + return Expressions.call(BeamExecUtil.class, MethodNames.TO_BYTE_BUFFER, toObjectType(expr)); + } + + Expression castExpr; + if (targetType.getSqlTypeName().getFamily() == SqlTypeFamily.NUMERIC && expr.getType() == String.class) { + // Convert from String to numbers + castExpr = Expressions.call(targetJavaType, MethodNames.VALUE_OF, expr); + } else { + // All remaining cases, rely on Calcite casting expression + castExpr = Types.castIfNecessary(targetJavaType, expr); + } + + final Primitive fromPrimitive = Primitive.ofBox(expr.getType()); + final Primitive toPrimitive = Primitive.ofBox(targetJavaType); + if (fromPrimitive != null && fromPrimitive.isNumeric() && toPrimitive != null && toPrimitive.isNumeric()) { + // If casting from one numeric to another numeric we need to have null safe protection for avoid + // NullPointerException when accessing the value of the expression. + // Example casting var1 of type Integer to Long: + // Long var2 = var1 != null ? var1.longValue() : null; + final Expression nullSafeExpr = buildNullSafeExpr(ImmutableList.of(expr)); + return Expressions.condition(nullSafeExpr, castExpr, Expressions.constant(null)); + } + return castExpr; + } + + /** + * Gets a field from an avro record. + * + * @param rowType Row type of the relational operator + * @param index Index of the field to access + * @param recordName Record variable name for the relational operator + * @return An {@link Expression} to represent a field of a record + */ + public static Expression getFieldExpression(RelDataType rowType, int index, Expression recordName) { + final String fieldName = RelDataTypeToAvro.toAvroQualifedName(rowType.getFieldNames().get(index)); + final Expression getExpr = Expressions.call(recordName, MethodNames.AVRO_GET, Expressions.constant(fieldName)); + return convertType(getExpr, rowType.getFieldList().get(index).getType()); + } + + /** + * Gets the string key expression for a list of key columns of a record. This expression will + * be used in key functions of Beam join or window APIs. + * + * @param keyColumns List of key columns + * @param recordType Relational type of the record + * @param recordName Record variable name + * @return Beam code expression representing the string key + */ + public static Expression getStringKeyExpr(Iterable keyColumns, RelDataType recordType, Expression recordName) { + if (!keyColumns.iterator().hasNext()) { + return Expressions.constant(null); + } + final List arguments = new ArrayList<>(); + arguments.add(recordName); + for (int i : keyColumns) { + final String fieldName = RelDataTypeToAvro.toAvroQualifedName(recordType.getFieldNames().get(i)); + arguments.add(Expressions.constant(fieldName)); + } + if (arguments.size() == 1) { + return arguments.get(0); + } + return Expressions.call(BeamExecUtil.class, MethodNames.BUILD_STRING_KEY, arguments); + } + + /** + * Constructs the key function for joins or aggregates. + * + * @param receiver PCollection expression to apply the key function + * @param inputRecord Input record expression used to construct key for each record + * @param keyExpr Function to compute key from inputRecord. Not that this function will use inputRecord as a param + * @return A {@link Expression} to represent the key function for joins or aggregates. + */ + public static Expression getBeamKeyFunc(Expression receiver, ParameterExpression inputRecord, Expression keyExpr) { + final Expression keyFunc = + RexBeamUtils.makeLambdaFunction(Types.of(SerializableFunction.class, GenericRecord.class, String.class), + String.class, ImmutableList.of(inputRecord), ImmutableList.of(Blocks.toFunctionBlock(keyExpr)), + new HashSet<>()); + final Expression withKeysExpr = Expressions.call(WithKeys.class, OF, keyFunc); + final Expression keyType = + Expressions.call(withKeysExpr, WITH_KEY_TYPE, Expressions.call(TypeDescriptors.class, STRINGS)); + return Expressions.call(receiver, Methods.P_COLLECTION_APPLY, keyType); + } + + + /** + * Maps from sqlKind operator to Java expression type. + * + * @param sqlKind sqlKind operator + * @return Java expression type + */ + static ExpressionType sqlKindToExprType(SqlKind sqlKind) { + switch (sqlKind) { + case AND: + return ExpressionType.AndAlso; + case OR: + return ExpressionType.OrElse; + case PLUS: + return ExpressionType.Add; + case MINUS: + return ExpressionType.Subtract; + case DIVIDE: + return ExpressionType.Divide; + case TIMES: + return ExpressionType.Multiply; + default: + throw new UnsupportedOperationException("Expression type: " + sqlKind + " not supported"); + } + } +} diff --git a/coral-beam/src/main/java/com/linkedin/beam/utils/RexToBeamConverter.java b/coral-beam/src/main/java/com/linkedin/beam/utils/RexToBeamConverter.java new file mode 100644 index 000000000..961c01413 --- /dev/null +++ b/coral-beam/src/main/java/com/linkedin/beam/utils/RexToBeamConverter.java @@ -0,0 +1,630 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.utils; + +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.operators.BeamNode; +import com.linkedin.beam.excution.BeamExecUtil; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericRecord; +import org.apache.calcite.linq4j.tree.Expression; +import org.apache.calcite.linq4j.tree.ExpressionType; +import org.apache.calcite.linq4j.tree.Expressions; +import org.apache.calcite.linq4j.tree.MethodCallExpression; +import org.apache.calcite.linq4j.tree.ParameterExpression; +import org.apache.calcite.linq4j.tree.Statement; +import org.apache.calcite.linq4j.tree.Types; +import org.apache.calcite.piglet.PigUserDefinedFunction; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexFieldAccess; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.schema.ScalarFunction; +import org.apache.calcite.schema.impl.ScalarFunctionImpl; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlSpecialOperator; +import org.apache.calcite.sql.fun.SqlRandFunction; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.validate.SqlUserDefinedFunction; +import org.apache.calcite.util.NlsString; +import org.apache.pig.data.DataByteArray; +import org.apache.pig.impl.util.avro.AvroBagWrapper; +import org.apache.pig.impl.util.avro.AvroTupleWrapper; + + +public class RexToBeamConverter { + private static final Expression TRUE = Expressions.constant(true); + // CONSTANT VARS: + // Pig UDF map: map each unique Pig UDF name to @PigUserDefinedFunction. + // Sever as constant input during lifetime of RexToBeam + private final Map pigUDFs; + + // GLOBAL VARS: + // Map to store any avro schemas that need to be declared and used in augogen code. + // This map may get changed after each call to convert() method. The final version of the map will + // be used to declared avro schemas in autogen code. + private Map registeredSchemas; + + // LOCAL VARS: + // Store all checked exceptions that can happen during evaluating an expression. Callers, after calling to + // convert() method, must call getLocalExeptions() to include code to handle all checked exceptions. + // Reset in each call to convert() method. + private Set localExeptions; + // Store map of temporary variable when evaluating an expression. That helps to reuse temporary results and make + // the autogen code shorter and more readable. + // Reset in each call to convert() method. + private Map localVarMap; + // Store declarations of temporary local variables when evaluating an expression. These local variables must be + // declared before the main expression in autogen code. Callers, after calling to convert() method, must call + // getLocalVarDeclarations() to get the list of local var declarations. + // Reset in each call to convert() method. + private List localVarDeclarations; + // Local variable counter, used to give different names for local vars. + // Should be reset before using local variables within a same scope + private int varCount; + + private boolean pigUDFContext = false; + private int constCount; + private String schemaName = null; + + public RexToBeamConverter(Map pigUDFs) { + this.pigUDFs = pigUDFs; + registeredSchemas = new HashMap<>(); + localExeptions = new HashSet<>(); + localVarMap = new HashMap<>(); + localVarDeclarations = new ArrayList<>(); + varCount = 0; + constCount = 0; + } + + public Set getLocalExeptions() { + return localExeptions; + } + + public List getLocalVarDeclarations() { + return localVarDeclarations; + } + + public Map getRegisteredSchemas() { + return registeredSchemas; + } + + private void resetLocalEnv(String outputSchemaName) { + localExeptions.clear(); + localVarMap.clear(); + localVarDeclarations.clear(); + schemaName = outputSchemaName; + } + + public void resetVarCounter() { + varCount = 0; + constCount = 0; + } + + /** + * Converts an relational expression into Beam code expression. + * + * @param rexNode The relational expression + * @param relNode The base relational operator for the relational expression + * @return Beam code expression + */ + public Expression convert(RexNode rexNode, RelNode relNode) { + return convert(rexNode, relNode, null); + } + + /** + * Converts an relational expression into Beam code expression. + * + * @param rexNode The relational expression + * @param relNode The base relational operator for the relational expression + * @param outputSchemaName The schema name of the expression. Ignore for null. + * @return Beam code expression + */ + public Expression convert(RexNode rexNode, RelNode relNode, String outputSchemaName) { + resetLocalEnv(outputSchemaName); + return convertInternal(rexNode, relNode); + } + + private Expression convertInternal(RexNode rexNode, RelNode relNode) { + final ParameterExpression localVar = localVarMap.get(rexNode); + if (localVar != null) { + // If the expression is computed and stored in an existing local variable, just use the local variable. + return localVar; + } + + switch (rexNode.getKind()) { + case FIELD_ACCESS: { + // For expression record.filedName + final RexFieldAccess fieldAccess = (RexFieldAccess) rexNode; + final Expression inputExpr = convertInternal(fieldAccess.getReferenceExpr(), relNode); + return declareLocalVar(rexNode, getInternalRecordField(inputExpr, fieldAccess)); + } + case INPUT_REF: { + final Expression outputExpr = RexBeamUtils.getFieldExpression(relNode.getRowType(), ((RexInputRef) rexNode).getIndex(), + BeamNode.getRecordVar((BeamNode) relNode)); + return declareLocalVar(rexNode, outputExpr); + } + case LITERAL: { + return convertLiteral((RexLiteral) rexNode, schemaName + "_CONST_" + constCount++); + } + case MINUS: + case PLUS: + case DIVIDE: + case TIMES: { + final List expOperands = + convertRexList(((RexCall) rexNode).getOperands(), relNode); + // Check whether any operand is null + final Expression nullSafeExpr = RexBeamUtils.buildNullSafeExpr(expOperands); + final Expression mainExpr = + Expressions.makeBinary(RexBeamUtils.sqlKindToExprType(rexNode.getKind()), expOperands.get(0), expOperands.get(1)); + final Expression outputExpr = Expressions.condition(nullSafeExpr, mainExpr, Expressions.constant(null)); + return declareLocalVar(rexNode, outputExpr); + } + case AND: + case OR: { + final List operands = + convertRexList(((RexCall) rexNode).getOperands(), relNode); + Expression retExpr = + Expressions.makeBinary(RexBeamUtils.sqlKindToExprType(rexNode.getKind()), operands.get(0), operands.get(1)); + for (int i = 2; i < operands.size(); i++) { + retExpr = Expressions.makeBinary(RexBeamUtils.sqlKindToExprType(rexNode.getKind()), retExpr, operands.get(i)); + } + return retExpr; + } + case EQUALS: + case NOT_EQUALS: { + final List operands = ((RexCall) rexNode).getOperands(); + final List expOperands = convertRexList(operands, relNode); + final Expression nullSafeExpr = RexBeamUtils.buildNullSafeExpr(expOperands); + Expression comparisionExpr; + if (operands.get(0).getType().getSqlTypeName().getFamily() == SqlTypeFamily.CHARACTER) { + // String comparison + final Method stringCompare = Types.lookupMethod(String.class, MethodNames.EQUALS, Object.class); + comparisionExpr = Expressions.call(expOperands.get(0), stringCompare, + RexBeamUtils.toJavaStringType(expOperands.get(1))); + if (rexNode.getKind() == SqlKind.NOT_EQUALS) { + comparisionExpr = Expressions.not(comparisionExpr); + } + } else { + // Other numeric comparisons. + if (rexNode.getKind() == SqlKind.EQUALS) { + comparisionExpr = Expressions.equal(expOperands.get(0), expOperands.get(1)); + } else { + comparisionExpr = Expressions.notEqual(expOperands.get(0), expOperands.get(1)); + } + } + if (comparisionExpr.equals(nullSafeExpr)) { + // Comparison is just a null check, avoid duplicate expression + return comparisionExpr; + } + return Expressions.andAlso(nullSafeExpr, comparisionExpr); + } + case LESS_THAN: + case LESS_THAN_OR_EQUAL: + case GREATER_THAN: + case GREATER_THAN_OR_EQUAL: { + final List operands = ((RexCall) rexNode).getOperands(); + final List expOperands = convertRexList(operands, relNode); + final Expression nullSafeExpr = RexBeamUtils.buildNullSafeExpr(expOperands); + Expression op1 = expOperands.get(0); + Expression op2 = expOperands.get(1); + if (operands.get(0).getType().getSqlTypeName().getFamily() == SqlTypeFamily.CHARACTER) { + op1 = RexBeamUtils.stringCompareExpr(expOperands.get(0), expOperands.get(1)); + op2 = Expressions.constant(0); + } + Expression comparisionExpr = null; + switch (rexNode.getKind()) { + case LESS_THAN: + comparisionExpr = Expressions.lessThan(op1, op2); + break; + case LESS_THAN_OR_EQUAL: + comparisionExpr = Expressions.lessThanOrEqual(op1, op2); + break; + case GREATER_THAN: + comparisionExpr = Expressions.greaterThan(op1, op2); + break; + case GREATER_THAN_OR_EQUAL: + comparisionExpr = Expressions.greaterThanOrEqual(op1, op2); + break; + default: + throw new UnsupportedOperationException("Expression type: " + rexNode.getKind() + " not supported"); + } + return Expressions.andAlso(nullSafeExpr, comparisionExpr); + } + case CASE: { + final List operands = + convertRexList(((RexCall) rexNode).getOperands(), relNode); + assert (operands.size() == 3); + return Expressions.condition(operands.get(0), operands.get(1), operands.get(2)); + } + case NOT: { + final Expression expr = convertRexList(((RexCall) rexNode).getOperands(), relNode).get(0); + final Expression nullSafeExpr = RexBeamUtils.buildNullSafeExpr(ImmutableList.of(expr)); + if (nullSafeExpr.equals(TRUE)) { + return Expressions.not(expr); + } + return Expressions.andAlso(nullSafeExpr, Expressions.not(expr)); + } + case IS_NULL: { + final Expression operand = + convertRexList(((RexCall) rexNode).getOperands(), relNode).get(0); + return Expressions.equal(operand, Expressions.constant(null)); + } + case IS_NOT_NULL: { + return RexBeamUtils.notNull(convertRexList(((RexCall) rexNode).getOperands(), relNode).get(0)); + } + case CAST: { + final List inputOperands = ((RexCall) rexNode).getOperands(); + final Expression inputOperandExpr = convertRexList(inputOperands, relNode).get(0); + final RelDataType targetType = rexNode.getType(); + + if (schemaName != null + && ((targetType.isStruct() || targetType.getComponentType() != null))) { + final ParameterExpression schemaExpr = registerSchema(schemaName, rexNode.getType()); + if (rexNode.getType().isStruct()) { + // Cast to record type + return toAvroRecord(inputOperandExpr, schemaExpr); + } else { + // Cast to array/multiset type + return toAvroArray(inputOperandExpr, schemaExpr); + } + } + return declareLocalVar(rexNode, RexBeamUtils.convertType(inputOperandExpr, rexNode.getType())); + } + + case OTHER_FUNCTION: { + final RexCall call = (RexCall) rexNode; + final SqlOperator op = call.getOperator(); + + if (op instanceof PigUserDefinedFunction) { + // Set Pig UDF context before converting UDF parameters + pigUDFContext = true; + final List exOperands = convertRexList(call.getOperands(), relNode); + final PigUserDefinedFunction pigFunc = (PigUserDefinedFunction) op; + assert pigFunc.getFunction() instanceof ScalarFunction; + final Method udfMethod = ((ScalarFunctionImpl) pigFunc.getFunction()).method; + final String udfKeyName = PigUDFUtils.getUDFMapKey(pigFunc); + final Expression udfCallExpr = callUDF(udfMethod, udfKeyName, exOperands); + try { + if (pigUDFs.containsKey(udfKeyName)) { + // If the return type is a struct or array, we need to convert data types + // from Pig tuple/databag to Avro record/array + final RelDataType returnType = rexNode.getType(); + if (returnType.isStruct() || returnType.getComponentType() != null) { + final ParameterExpression schemaExpr = + registerSchema(RexBeamUtils.getSchemaName(udfKeyName), returnType); + final ParameterExpression udfCallVar = declareLocalVar(null, udfCallExpr); + final Expression toAvroVar = + returnType.isStruct() ? toAvroRecord(udfCallVar, schemaExpr) : toAvroArray(udfCallVar, schemaExpr); + return declareLocalVar(null, toAvroVar); + } + } + return declareLocalVar(rexNode, udfCallExpr); + } finally { + pigUDFContext = false; + } + } + + final List exOperands = convertRexList(call.getOperands(), relNode); + if (op instanceof SqlSpecialOperator) { + switch (op.getName()) { + case "ITEM": + return RexBeamUtils.convertType( + Expressions.call(exOperands.get(0), MethodNames.MAP_GET, Expressions.convert_(exOperands.get(1), Object.class)), + call.getType()); + default: + throw new UnsupportedOperationException("SQL Operator: " + op.getName() + " not supported"); + } + } + + if (op instanceof SqlRandFunction) { + return declareLocalVar(null, Expressions.call(BeamExecUtil.class, MethodNames.RAND, ImmutableList.of())); + } + + if (op instanceof SqlFunction) { + final List operands = + convertRexList(((RexCall) rexNode).getOperands(), relNode); + switch (op.getName()) { + case "UPPER": + case "LOWER": { + final Expression nullSafeExpr = RexBeamUtils.buildNullSafeExpr(operands); + final Expression toUpperCaseExpr = Expressions.call(operands.get(0), + op.getName().equals("UPPER") ? MethodNames.TO_UPPER : MethodNames.TO_LOWER); + final Expression conditionExpr = + Expressions.condition(nullSafeExpr, toUpperCaseExpr, Expressions.constant(null)); + return declareLocalVar(null, conditionExpr); + } + case "TIME_ROUND_UP": + case "DATE_FORMAT": { + final SqlUserDefinedFunction func = (SqlUserDefinedFunction) op; + final Method udfMethod = ((ScalarFunctionImpl) func.getFunction()).method; + final Expression funcCallExpr = Expressions.call(udfMethod, operands); + return declareLocalVar(null, funcCallExpr); + } + default: + break; + } + } + throw new UnsupportedOperationException("SQL Operator: " + op.getName() + " not supported"); + } + case LIKE: { + final List operands = + convertRexList(((RexCall) rexNode).getOperands(), relNode); + assert operands.size() == 2; + // Need to swap the operand order before calling Java Pattern.matches + return Expressions.call(Pattern.class, MethodNames.MATCHES, + ImmutableList.of(operands.get(1), operands.get(0))); + } + case OTHER: { + if (rexNode instanceof RexCall) { + final RexCall call = (RexCall) rexNode; + final List exOperands = convertRexList(call.getOperands(), relNode); + final SqlOperator op = ((RexCall) rexNode).getOperator(); + if (op == SqlStdOperatorTable.CONCAT) { + if (call.getType().getSqlTypeName().getFamily() == SqlTypeFamily.CHARACTER) { + return Expressions.makeBinary(ExpressionType.Add, exOperands.get(0), exOperands.get(1)); + } else { + throw new UnsupportedOperationException("CONCAT on type: " + call.getType().getSqlTypeName() + " not supported"); + } + } + } + throw new UnsupportedOperationException("OTHER expression: " + rexNode.toString() + " not supported"); + } + default: + throw new UnsupportedOperationException("Expression type: " + rexNode.getKind() + " not supported"); + } + } + + public Expression adjustConstant(Expression source, RelDataType type, String schemaName) { + switch (type.getSqlTypeName()) { + case ROW: { + final ParameterExpression schemaExpr = registerSchema(schemaName, type); + return toAvroRecord(source, schemaExpr); + } + case MULTISET: { + final ParameterExpression schemaExpr = registerSchema(schemaName, type); + if (type.getComponentType().isStruct()) { + final MethodCallExpression callExpr = (MethodCallExpression) source; + final List newExprs = new ArrayList<>(); + for (Expression childExpr : callExpr.expressions) { + newExprs.add(adjustConstant(childExpr, type.getComponentType(), schemaName + "_COMPONENT")); + } + callExpr.expressions.clear(); + callExpr.expressions.addAll(newExprs); + } + return toAvroArray(source, schemaExpr); + } + default: + return source; + } + + } + + /** + * Converts a relational literal into a Java code constant expression + * + * TODO This method should be aware of whether the literal is for PigUDF (then Pig constant) or not (Avro constants) + */ + Expression convertLiteral(RexLiteral literal, String schemaName) { + if (literal.getValue() == null) { + return Expressions.constant(null); + } + switch (literal.getType().getSqlTypeName()) { + case CHAR: + case VARCHAR: + return Expressions.constant(((NlsString) literal.getValue()).getValue(), String.class); + case INTEGER: + return Expressions.constant(((BigDecimal) literal.getValue()).intValue(), int.class); + case BIGINT: + return Expressions.constant(((BigDecimal) literal.getValue()).longValue(), long.class); + case DOUBLE: + return Expressions.constant(((BigDecimal) literal.getValue()).doubleValue(), double.class); + case BOOLEAN: + return Expressions.constant(literal.getValue(), boolean.class); + case ROW: { + if (!(literal.getValue() instanceof List)) { + throw new UnsupportedOperationException("Record constant must be a list"); + } + final List literals = (List) literal.getValue(); + final List literalExprs = new ArrayList<>(); + for (RexLiteral rexLiteral : literals) { + literalExprs.add(convertLiteral(rexLiteral, schemaName)); + } + final Expression pigTuple = Expressions.call(Methods.BUILD_TUPLE, literalExprs); + if (pigUDFContext) { + return pigTuple; + } + final ParameterExpression schemaExpr = registerSchema(schemaName, literal.getType()); + return toAvroRecord(pigTuple, schemaExpr); + } + case MULTISET: { + if (!(literal.getValue() instanceof List)) { + throw new UnsupportedOperationException("Multiset constant must be a list"); + } + final List literals = (List) literal.getValue(); + final List literalExprs = new ArrayList<>(); + final String componentSchema = schemaName + "_COMP"; + for (RexLiteral rexLiteral : literals) { + literalExprs.add(convertLiteral(rexLiteral, componentSchema)); + } + + final Expression pigDatabag = Expressions.call(Methods.BUILD_BAG, literalExprs); + if (pigUDFContext) { + return pigDatabag; + } + final ParameterExpression schemaExpr = registerSchema(schemaName, literal.getType()); + return toAvroArray(pigDatabag, schemaExpr); + } + default: + throw new UnsupportedOperationException( + "constant type: " + literal.getType().getSqlTypeName() + " not supported"); + } + } + + /** + * Converts a list of relational expressions into a list of Beam code expressions. + * + * @param rexNodes The list of relational expressions + * @param relNode The base relational operator for all relational expressions + */ + private List convertRexList(List rexNodes, RelNode relNode) { + List expressions = new ArrayList<>(); + for (RexNode operand : rexNodes) { + expressions.add(convertInternal(operand, relNode)); + } + return expressions; + } + + /** + * Registers avro schema for record or array types. + * + * @param schemaName local variable name for the schema + * @param relType Rel type of the schema + * @return Parameter expression for the registered schema + */ + public ParameterExpression registerSchema(String schemaName, RelDataType relType) { + final RelDataType existingType = registeredSchemas.get(schemaName); + String registeredName = schemaName; + if (existingType != null && !existingType.equals(relType)) { + registeredName = schemaName + (registeredSchemas.size() + 1); + } + registeredSchemas.put(registeredName, relType); + return RexBeamUtils.getSchemaParam(registeredName); + } + + /** + * Converts an expression to avro array if required. + * + * @param expr input expression + * @param schemaExpr schema for the input expression + * @return Expression with avro array type + */ + private Expression toAvroArray(Expression expr, ParameterExpression schemaExpr) { + if (expr.getType() == GenericArray.class) { + return expr; + } + return Expressions.call(BeamExecUtil.class, MethodNames.TO_AVRO_ARRAY, ImmutableList.of(expr, schemaExpr)); + } + + /** + * Converts an expression to avro record if required. + * + * @param expr input expression + * @param schemaExpr schema for the input expression + * @return Expression with avro record type + */ + private Expression toAvroRecord(Expression expr, ParameterExpression schemaExpr) { + if (expr.getType() == GenericRecord.class) { + return expr; + } + return Expressions.call(BeamExecUtil.class, MethodNames.TO_AVRO_RECORD, ImmutableList.of(expr, schemaExpr)); + } + + /** + * Declares a local variables for an expression. + * + * @param node Original relational expression + * @param assignedExpression Beam code expression + * @return Local variable expression representing the expression + */ + private ParameterExpression declareLocalVar(RexNode node, Expression assignedExpression) { + if (assignedExpression instanceof ParameterExpression) { + // If the assigned expression is already a variable, no need to declare a new variable + return (ParameterExpression) assignedExpression; + } + + final ParameterExpression localVar = Expressions.parameter(assignedExpression.getType(), "var" + varCount++); + localVarDeclarations.add(Expressions.declare(Modifier.FINAL, localVar, assignedExpression)); + if (node != null) { + localVarMap.put(node, localVar); + } + return localVar; + } + + /** + * Calls an UDF. + * + * @param udfMethod Method implementing the UDF + * @param pigUDFKey Key to lookup Pig UDF map for pre-initialized Pig UDF object + * @param exOperands UDF operands + */ + private Expression callUDF(Method udfMethod, String pigUDFKey, List exOperands) { + localExeptions.addAll(Arrays.asList(udfMethod.getExceptionTypes())); + final Class udfClass = udfMethod.getDeclaringClass(); + final PigUserDefinedFunction pigFunc = pigUDFs.get(pigUDFKey); + if (pigFunc != null) { + // Pig UDF was already declared and initialized in the beginning of the code-gen file + return Expressions.call(PigUDFUtils.getPigUDFParamExpr(pigFunc), MethodNames.EXEC, exOperands); + } else if (Modifier.isStatic(udfMethod.getModifiers())) { + List pigExpOperands = exOperands; + if (udfClass.getName().equals("org.apache.calcite.piglet.PigRelSqlUdfs") && udfMethod.getName() + .equals(MethodNames.BUILD_TUPLE)) { + // To buil Pig Tuple for the UDF, need to wrap operands into Pig types + pigExpOperands = wrapPigOperands(exOperands); + } + return Expressions.call(udfMethod, pigExpOperands); + } else { + return Expressions.call(Expressions.new_(udfMethod.getDeclaringClass()), udfMethod.getName(), exOperands); + } + } + + /** + * If input for buildTuple() is in Avro types, need to wrap it into Pig types + */ + private List wrapPigOperands(List exOperands) { + final List newOperands = new ArrayList<>(); + for (Expression operand : exOperands) { + if (operand.getType() == GenericArray.class) { + newOperands.add(declareLocalVar(null, Expressions.new_(AvroBagWrapper.class, operand))); + } else if (operand.getType() == GenericRecord.class) { + // TODO: GenericRecord to Pig tuple not tested + newOperands.add(declareLocalVar(null, Expressions.new_(AvroTupleWrapper.class, operand))); + } else if (operand.getType() == ByteBuffer.class) { + final Expression nullSafeExpr = RexBeamUtils.buildNullSafeExpr(ImmutableList.of(operand)); + final Expression byteBufferArrayExpr = + Expressions.new_(DataByteArray.class, Expressions.call(operand, MethodNames.ARRAY)); + final Expression dataByteArrayExpr = + Expressions.condition(nullSafeExpr, byteBufferArrayExpr, Expressions.constant(null)); + newOperands.add(declareLocalVar(null, dataByteArrayExpr)); + } else { + newOperands.add(operand); + } + } + return newOperands; + } + + /** + * Gets a field of an internal record. + * + * @param inputExpr Expression representing the record + * @param fieldAccess Field to access + */ + private Expression getInternalRecordField(Expression inputExpr, RexFieldAccess fieldAccess) { + final Expression nullSafeExpr = RexBeamUtils.buildNullSafeExpr(ImmutableList.of(inputExpr)); + final Expression result = RexBeamUtils.convertType(Expressions.call(inputExpr, MethodNames.AVRO_GET, + Expressions.constant(RelDataTypeToAvro.toAvroQualifedName(fieldAccess.getField().getName()))), + fieldAccess.getType()); + return Expressions.condition(nullSafeExpr, result, Expressions.constant(null)); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/AggregateTestApplication.java b/coral-beam/src/test/beamGeneratedCode/AggregateTestApplication.java new file mode 100644 index 000000000..4beb41a35 --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/AggregateTestApplication.java @@ -0,0 +1,258 @@ +import com.linkedin.beam.excution.BeamExecUtil; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.Objects; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.commons.lang.ObjectUtils; +import org.joda.time.Duration; + +public class AggregateTestApplication { + private static final Schema PROJECT2_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"strGroupCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"longGroupCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"longCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"intCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"doubleCol\",\"type\":[\"double\",\"null\"]},{\"name\":\"stringCol\",\"type\":[\"string\",\"null\"]}]}"); + private static final Schema AGGREGATE3_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"strGroupCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"longGroupCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"_f2\",\"type\":[\"long\",\"null\"]},{\"name\":\"_f3\",\"type\":[\"int\",\"null\"]},{\"name\":\"_f4\",\"type\":[\"double\",\"null\"]},{\"name\":\"_f5\",\"type\":[\"string\",\"null\"]},{\"name\":\"_f6\",\"type\":[\"long\",\"null\"]},{\"name\":\"_f7\",\"type\":[\"int\",\"null\"]},{\"name\":\"_f8\",\"type\":[\"double\",\"null\"]},{\"name\":\"_f9\",\"type\":[\"string\",\"null\"]},{\"name\":\"_f10\",\"type\":[\"long\",\"null\"]},{\"name\":\"_f11\",\"type\":[\"int\",\"null\"]},{\"name\":\"_f12\",\"type\":[\"double\",\"null\"]},{\"name\":\"_f13\",\"type\":\"long\"},{\"name\":\"_f14\",\"type\":\"long\"},{\"name\":\"_f15\",\"type\":{\"type\":\"array\",\"items\":[\"long\",\"null\"]}}]}"); + private static final Schema PROJECT4_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"strGroupCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"longGroupCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"maxLong\",\"type\":[\"long\",\"null\"]},{\"name\":\"maxInt\",\"type\":[\"int\",\"null\"]},{\"name\":\"maxDouble\",\"type\":[\"double\",\"null\"]},{\"name\":\"maxString\",\"type\":[\"string\",\"null\"]},{\"name\":\"minLong\",\"type\":[\"long\",\"null\"]},{\"name\":\"minInt\",\"type\":[\"int\",\"null\"]},{\"name\":\"minDouble\",\"type\":[\"double\",\"null\"]},{\"name\":\"minString\",\"type\":[\"string\",\"null\"]},{\"name\":\"sumLong\",\"type\":[\"long\",\"null\"]},{\"name\":\"sumInt\",\"type\":[\"long\",\"null\"]},{\"name\":\"sumDouble\",\"type\":[\"double\",\"null\"]},{\"name\":\"countLong\",\"type\":[\"long\",\"null\"]},{\"name\":\"countStar\",\"type\":[\"long\",\"null\"]},{\"name\":\"longMultiset\",\"type\":{\"type\":\"array\",\"items\":[\"long\",\"null\"]}}]}"); + private static final Schema AGGREGATE3_F15_SCHEMA = + new Schema.Parser().parse("{\"type\":\"array\",\"items\":[\"long\",\"null\"]}"); + + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read testStream", + KafkaIOGenericRecord.read().withTopic("testStream").withoutMetadata()); + PCollection> project2 = + stream1 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream1KV) { + final GenericRecord stream1Record = stream1KV.getValue(); + GenericRecord project2Record = new GenericData.Record(PROJECT2_SCHEMA); + final String var0 = + Objects.toString(stream1Record.get("strGroupCol"), null); + project2Record.put("strGroupCol", var0); + final Long var1 = (Long) stream1Record.get("longGroupCol"); + project2Record.put("longGroupCol", var1); + final Long var2 = (Long) stream1Record.get("longCol"); + project2Record.put("longCol", var2); + final Integer var3 = (Integer) stream1Record.get("intCol"); + project2Record.put("intCol", var3); + final Double var4 = (Double) stream1Record.get("doubleCol"); + project2Record.put("doubleCol", var4); + final String var5 = Objects.toString(stream1Record.get("stringCol"), null); + project2Record.put("stringCol", var5); + return KV.of(stream1KV.getKey(), project2Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT2_SCHEMA))); + PCollection> aggregate3KV = + project2 + .apply(Values.create()) + .apply( + WithKeys.of( + new SerializableFunction() { + public String apply(GenericRecord project2Record) { + { + return BeamExecUtil.buildStringKey( + project2Record, "strGroupCol", "longGroupCol"); + } + } + }) + .withKeyType(TypeDescriptors.strings())) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5L)))); + PCollection> aggregate3 = + aggregate3KV + .apply( + Combine.perKey( + new SerializableFunction, GenericRecord>() { + public GenericRecord apply(Iterable project2GroupedRecords) { + GenericRecord aggregate3Record = new GenericData.Record(AGGREGATE3_SCHEMA); + Iterator iterator = project2GroupedRecords.iterator(); + GenericRecord inputRecord = iterator.next(); + aggregate3Record.put( + "strGroupCol", + (Object) Objects.toString(inputRecord.get("strGroupCol"), null)); + aggregate3Record.put("longGroupCol", inputRecord.get("longGroupCol")); + long aggregate3_f2 = (Long) inputRecord.get("longCol"); + int aggregate3_f3 = (Integer) inputRecord.get("intCol"); + double aggregate3_f4 = (Double) inputRecord.get("doubleCol"); + String aggregate3_f5 = Objects.toString(inputRecord.get("stringCol"), null); + long aggregate3_f6 = (Long) inputRecord.get("longCol"); + int aggregate3_f7 = (Integer) inputRecord.get("intCol"); + double aggregate3_f8 = (Double) inputRecord.get("doubleCol"); + String aggregate3_f9 = Objects.toString(inputRecord.get("stringCol"), null); + long aggregate3_f10 = 0; + int aggregate3_f11 = 0; + double aggregate3_f12 = 0; + long aggregate3_f13 = 0; + long aggregate3_f14 = 0; + GenericArray aggregate3_f15 = + new GenericData.Array(AGGREGATE3_F15_SCHEMA, new ArrayList()); + boolean doLoop = true; + while (doLoop) { + Long project2longCol = (Long) inputRecord.get("longCol"); + Integer project2intCol = (Integer) inputRecord.get("intCol"); + Double project2doubleCol = (Double) inputRecord.get("doubleCol"); + String project2stringCol = + Objects.toString(inputRecord.get("stringCol"), null); + if (project2longCol != null) { + aggregate3_f2 = + aggregate3_f2 > project2longCol ? aggregate3_f2 : project2longCol; + } + if (project2intCol != null) { + aggregate3_f3 = + aggregate3_f3 > project2intCol ? aggregate3_f3 : project2intCol; + } + if (project2doubleCol != null) { + aggregate3_f4 = + aggregate3_f4 > project2doubleCol + ? aggregate3_f4 + : project2doubleCol; + } + if (project2stringCol != null) { + aggregate3_f5 = + ObjectUtils.compare(aggregate3_f5, project2stringCol) > 0 + ? aggregate3_f5 + : project2stringCol; + } + if (project2longCol != null) { + aggregate3_f6 = + aggregate3_f6 < project2longCol ? aggregate3_f6 : project2longCol; + } + if (project2intCol != null) { + aggregate3_f7 = + aggregate3_f7 < project2intCol ? aggregate3_f7 : project2intCol; + } + if (project2doubleCol != null) { + aggregate3_f8 = + aggregate3_f8 < project2doubleCol + ? aggregate3_f8 + : project2doubleCol; + } + if (project2stringCol != null) { + aggregate3_f9 = + ObjectUtils.compare(aggregate3_f9, project2stringCol) < 0 + ? aggregate3_f9 + : project2stringCol; + } + if (project2longCol != null) { + aggregate3_f10 = aggregate3_f10 + project2longCol; + } + if (project2intCol != null) { + aggregate3_f11 = aggregate3_f11 + project2intCol; + } + if (project2doubleCol != null) { + aggregate3_f12 = aggregate3_f12 + project2doubleCol; + } + if (project2longCol != null) { + aggregate3_f13 = aggregate3_f13 + 1; + } + aggregate3_f14 = aggregate3_f14 + 1; + aggregate3_f15.add(project2longCol); + if (iterator.hasNext()) { + inputRecord = iterator.next(); + } else { + doLoop = false; + } + } + aggregate3Record.put("_f2", (Object) aggregate3_f2); + aggregate3Record.put("_f3", (Object) aggregate3_f3); + aggregate3Record.put("_f4", (Object) aggregate3_f4); + aggregate3Record.put("_f5", (Object) aggregate3_f5); + aggregate3Record.put("_f6", (Object) aggregate3_f6); + aggregate3Record.put("_f7", (Object) aggregate3_f7); + aggregate3Record.put("_f8", (Object) aggregate3_f8); + aggregate3Record.put("_f9", (Object) aggregate3_f9); + aggregate3Record.put("_f10", (Object) aggregate3_f10); + aggregate3Record.put("_f11", (Object) aggregate3_f11); + aggregate3Record.put("_f12", (Object) aggregate3_f12); + aggregate3Record.put("_f13", (Object) aggregate3_f13); + aggregate3Record.put("_f14", (Object) aggregate3_f14); + aggregate3Record.put("_f15", (Object) aggregate3_f15); + return aggregate3Record; + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(AGGREGATE3_SCHEMA))) + .apply(Window.into(new GlobalWindows())); + PCollection> project4 = + aggregate3 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply( + KV aggregate3KV) { + final GenericRecord aggregate3Record = aggregate3KV.getValue(); + GenericRecord project4Record = new GenericData.Record(PROJECT4_SCHEMA); + final String var0 = + Objects.toString(aggregate3Record.get("strGroupCol"), null); + project4Record.put("strGroupCol", var0); + final Long var1 = (Long) aggregate3Record.get("longGroupCol"); + project4Record.put("longGroupCol", var1); + final Long var2 = (Long) aggregate3Record.get("_f2"); + project4Record.put("maxLong", var2); + final Integer var3 = (Integer) aggregate3Record.get("_f3"); + project4Record.put("maxInt", var3); + final Double var4 = (Double) aggregate3Record.get("_f4"); + project4Record.put("maxDouble", var4); + final String var5 = Objects.toString(aggregate3Record.get("_f5"), null); + project4Record.put("maxString", var5); + final Long var6 = (Long) aggregate3Record.get("_f6"); + project4Record.put("minLong", var6); + final Integer var7 = (Integer) aggregate3Record.get("_f7"); + project4Record.put("minInt", var7); + final Double var8 = (Double) aggregate3Record.get("_f8"); + project4Record.put("minDouble", var8); + final String var9 = Objects.toString(aggregate3Record.get("_f9"), null); + project4Record.put("minString", var9); + final Long var10 = (Long) aggregate3Record.get("_f10"); + project4Record.put("sumLong", var10); + final Integer var11 = (Integer) aggregate3Record.get("_f11"); + final Long var12 = var11 != null ? var11.longValue() : (Long) null; + project4Record.put("sumInt", var12); + final Double var13 = (Double) aggregate3Record.get("_f12"); + project4Record.put("sumDouble", var13); + final Long var14 = (Long) aggregate3Record.get("_f13"); + project4Record.put("countLong", var14); + final Long var15 = (Long) aggregate3Record.get("_f14"); + project4Record.put("countStar", var15); + final GenericArray var16 = (GenericArray) aggregate3Record.get("_f15"); + project4Record.put("longMultiset", var16); + return KV.of(aggregate3KV.getKey(), project4Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT4_SCHEMA))); + project4.apply(KafkaIOGenericRecord.write().withTopic("aggregate_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/ArrayFlatten1CTestApplication.java b/coral-beam/src/test/beamGeneratedCode/ArrayFlatten1CTestApplication.java new file mode 100644 index 000000000..ef6b6eaa5 --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/ArrayFlatten1CTestApplication.java @@ -0,0 +1,92 @@ +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.excution.BeamArrayFlatten; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import java.util.Collection; +import java.util.Objects; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.FlatMapElements; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +public class ArrayFlatten1CTestApplication { + private static final Schema PROJECT2_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"key\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[{\"type\":\"array\",\"items\":[{\"type\":\"record\",\"name\":\"record0_longCol\",\"fields\":[{\"name\":\"elem\",\"type\":[\"long\",\"null\"]}]},\"null\"]},\"null\"]}]}"); + private static final Schema ARRAYFLATTEN3_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"key\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[{\"type\":\"array\",\"items\":[{\"type\":\"record\",\"name\":\"record0_longCol\",\"fields\":[{\"name\":\"elem\",\"type\":[\"long\",\"null\"]}]},\"null\"]},\"null\"]},{\"name\":\"elem\",\"type\":[\"long\",\"null\"]}]}"); + private static final Schema PROJECT4_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"key\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[\"long\",\"null\"]}]}"); + + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read arrayFlattenStream", + KafkaIOGenericRecord.read().withTopic("arrayFlattenStream").withoutMetadata()); + PCollection> project2 = + stream1 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream1KV) { + final GenericRecord stream1Record = stream1KV.getValue(); + GenericRecord project2Record = new GenericData.Record(PROJECT2_SCHEMA); + final String var0 = Objects.toString(stream1Record.get("key"), null); + project2Record.put("key", var0); + final GenericArray var1 = (GenericArray) stream1Record.get("longArray"); + project2Record.put("longCol", var1); + return KV.of(stream1KV.getKey(), project2Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT2_SCHEMA))); + PCollection> arrayflatten3 = + project2 + .apply( + FlatMapElements.via( + new SimpleFunction< + KV, Collection>>() { + public Collection> apply( + KV project2KV) { + return BeamArrayFlatten.flatten( + project2KV, ImmutableList.of("longCol"), ARRAYFLATTEN3_SCHEMA); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(ARRAYFLATTEN3_SCHEMA))); + PCollection> project4 = + arrayflatten3 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply( + KV arrayflatten3KV) { + final GenericRecord arrayflatten3Record = arrayflatten3KV.getValue(); + GenericRecord project4Record = new GenericData.Record(PROJECT4_SCHEMA); + final String var0 = Objects.toString(arrayflatten3Record.get("key"), null); + project4Record.put("key", var0); + final Long var1 = (Long) arrayflatten3Record.get("elem"); + project4Record.put("longCol", var1); + return KV.of(arrayflatten3KV.getKey(), project4Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT4_SCHEMA))); + project4.apply(KafkaIOGenericRecord.write().withTopic("array_flatten1_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/ArrayFlatten2CTestApplication.java b/coral-beam/src/test/beamGeneratedCode/ArrayFlatten2CTestApplication.java new file mode 100644 index 000000000..a8e190305 --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/ArrayFlatten2CTestApplication.java @@ -0,0 +1,98 @@ +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.excution.BeamArrayFlatten; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import java.util.Collection; +import java.util.Objects; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericArray; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.FlatMapElements; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +public class ArrayFlatten2CTestApplication { + private static final Schema PROJECT2_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"key\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[{\"type\":\"array\",\"items\":[{\"type\":\"record\",\"name\":\"record0_longCol\",\"fields\":[{\"name\":\"lElem\",\"type\":[\"long\",\"null\"]}]},\"null\"]},\"null\"]},{\"name\":\"doubleCol\",\"type\":[{\"type\":\"array\",\"items\":[{\"type\":\"record\",\"name\":\"record0_doubleCol\",\"fields\":[{\"name\":\"dElem\",\"type\":[\"double\",\"null\"]}]},\"null\"]},\"null\"]}]}"); + private static final Schema ARRAYFLATTEN3_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"key\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[{\"type\":\"array\",\"items\":[{\"type\":\"record\",\"name\":\"record0_longCol\",\"fields\":[{\"name\":\"lElem\",\"type\":[\"long\",\"null\"]}]},\"null\"]},\"null\"]},{\"name\":\"doubleCol\",\"type\":[{\"type\":\"array\",\"items\":[{\"type\":\"record\",\"name\":\"record0_doubleCol\",\"fields\":[{\"name\":\"dElem\",\"type\":[\"double\",\"null\"]}]},\"null\"]},\"null\"]},{\"name\":\"lElem\",\"type\":[\"long\",\"null\"]},{\"name\":\"dElem\",\"type\":[\"double\",\"null\"]}]}"); + private static final Schema PROJECT4_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"key\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"doubleCol\",\"type\":[\"double\",\"null\"]}]}"); + + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read arrayFlattenStream", + KafkaIOGenericRecord.read().withTopic("arrayFlattenStream").withoutMetadata()); + PCollection> project2 = + stream1 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream1KV) { + final GenericRecord stream1Record = stream1KV.getValue(); + GenericRecord project2Record = new GenericData.Record(PROJECT2_SCHEMA); + final String var0 = Objects.toString(stream1Record.get("key"), null); + project2Record.put("key", var0); + final GenericArray var1 = (GenericArray) stream1Record.get("longArray"); + project2Record.put("longCol", var1); + final GenericArray var2 = (GenericArray) stream1Record.get("doubleArray"); + project2Record.put("doubleCol", var2); + return KV.of(stream1KV.getKey(), project2Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT2_SCHEMA))); + PCollection> arrayflatten3 = + project2 + .apply( + FlatMapElements.via( + new SimpleFunction< + KV, Collection>>() { + public Collection> apply( + KV project2KV) { + return BeamArrayFlatten.flatten( + project2KV, + ImmutableList.of("longCol", "doubleCol"), + ARRAYFLATTEN3_SCHEMA); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(ARRAYFLATTEN3_SCHEMA))); + PCollection> project4 = + arrayflatten3 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply( + KV arrayflatten3KV) { + final GenericRecord arrayflatten3Record = arrayflatten3KV.getValue(); + GenericRecord project4Record = new GenericData.Record(PROJECT4_SCHEMA); + final String var0 = Objects.toString(arrayflatten3Record.get("key"), null); + project4Record.put("key", var0); + final Long var1 = (Long) arrayflatten3Record.get("lElem"); + project4Record.put("longCol", var1); + final Double var2 = (Double) arrayflatten3Record.get("dElem"); + project4Record.put("doubleCol", var2); + return KV.of(arrayflatten3KV.getKey(), project4Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT4_SCHEMA))); + project4.apply(KafkaIOGenericRecord.write().withTopic("array_flatten1_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/DistinctTestApplication.java b/coral-beam/src/test/beamGeneratedCode/DistinctTestApplication.java new file mode 100644 index 000000000..664d466ad --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/DistinctTestApplication.java @@ -0,0 +1,42 @@ +import com.linkedin.beam.excution.BeamExecUtil; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import com.linkedin.beam.excution.MessageDedup; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.joda.time.Duration; + +public class DistinctTestApplication { + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read testStream", + KafkaIOGenericRecord.read().withTopic("testStream").withoutMetadata()); + PCollection> aggregate2 = + stream1 + .apply(Values.create()) + .apply( + WithKeys.of( + new SerializableFunction() { + public String apply(GenericRecord stream1Record) { + { + return BeamExecUtil.buildDistinctStringKeyFromRecord( + "aggregate2_", stream1Record, "timestamp"); + } + } + }) + .withKeyType(TypeDescriptors.strings())) + .apply(MessageDedup.within(Duration.standardDays(1))); + aggregate2.apply(KafkaIOGenericRecord.write().withTopic("distinct_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/FilterTestApplication.java b/coral-beam/src/test/beamGeneratedCode/FilterTestApplication.java new file mode 100644 index 000000000..4c759ecfb --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/FilterTestApplication.java @@ -0,0 +1,38 @@ +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import java.util.Objects; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Filter; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +public class FilterTestApplication { + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read testStream", + KafkaIOGenericRecord.read().withTopic("testStream").withoutMetadata()); + PCollection> filter2 = + stream1.apply( + Filter.by( + new SerializableFunction, Boolean>() { + public Boolean apply(KV stream1KV) { + final GenericRecord stream1Record = stream1KV.getValue(); + final Integer var0 = (Integer) stream1Record.get("intCol"); + final Double var1 = (Double) stream1Record.get("doubleCol"); + final Double var2 = var1 != null ? var1 * var1 : (Double) null; + final String var3 = Objects.toString(stream1Record.get("stringCol"), null); + return var0 != null + && var0 > 2 + && (var2 != null && var2 < 1.8D || var3 != null && var3.equals("test")); + } + })); + filter2.apply(KafkaIOGenericRecord.write().withTopic("filter_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/JoinTestApplication.java b/coral-beam/src/test/beamGeneratedCode/JoinTestApplication.java new file mode 100644 index 000000000..6a9ba247a --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/JoinTestApplication.java @@ -0,0 +1,139 @@ +import com.linkedin.beam.excution.BeamExecUtil; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import java.util.Objects; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.transforms.Join; +import org.apache.beam.sdk.schemas.utils.AvroUtils; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.joda.time.Duration; + +public class JoinTestApplication { + private static final Schema PROJECT2_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"intCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"doubleCol\",\"type\":[\"double\",\"null\"]}]}"); + private static final Schema PROJECT4_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"intCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[\"long\",\"null\"]}]}"); + private static final Schema JOIN5_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"intCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"doubleCol\",\"type\":[\"double\",\"null\"]},{\"name\":\"intCol0\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringCol0\",\"type\":[\"string\",\"null\"]},{\"name\":\"longCol\",\"type\":[\"long\",\"null\"]}]}"); + + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read joinStream", + KafkaIOGenericRecord.read().withTopic("joinStream").withoutMetadata()); + PCollection> project2 = + stream1 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream1KV) { + final GenericRecord stream1Record = stream1KV.getValue(); + GenericRecord project2Record = new GenericData.Record(PROJECT2_SCHEMA); + final Integer var0 = (Integer) stream1Record.get("intCol"); + project2Record.put("intCol", var0); + final String var1 = Objects.toString(stream1Record.get("stringCol"), null); + project2Record.put("stringCol", var1); + final Double var2 = (Double) stream1Record.get("doubleCol"); + project2Record.put("doubleCol", var2); + return KV.of(stream1KV.getKey(), project2Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT2_SCHEMA))); + PCollection> stream3 = + pipeline.apply( + "Read joinStream2", + KafkaIOGenericRecord.read().withTopic("joinStream2").withoutMetadata()); + PCollection> project4 = + stream3 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream3KV) { + final GenericRecord stream3Record = stream3KV.getValue(); + GenericRecord project4Record = new GenericData.Record(PROJECT4_SCHEMA); + final Integer var0 = (Integer) stream3Record.get("intCol"); + project4Record.put("intCol", var0); + final String var1 = Objects.toString(stream3Record.get("stringCol"), null); + project4Record.put("stringCol", var1); + final Long var2 = (Long) stream3Record.get("longCol"); + project4Record.put("longCol", var2); + return KV.of(stream3KV.getKey(), project4Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT4_SCHEMA))); + PCollection> join5Left = + project2 + .apply(Values.create()) + .apply( + WithKeys.of( + new SerializableFunction() { + public String apply(GenericRecord project2Record) { + { + return BeamExecUtil.buildStringKey( + project2Record, "intCol", "stringCol"); + } + } + }) + .withKeyType(TypeDescriptors.strings())) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5L)))); + PCollection> join5Right = + project4 + .apply(Values.create()) + .apply( + WithKeys.of( + new SerializableFunction() { + public String apply(GenericRecord project4Record) { + { + return BeamExecUtil.buildStringKey( + project4Record, "intCol", "stringCol"); + } + } + }) + .withKeyType(TypeDescriptors.strings())) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5L)))); + PCollection> join5 = + join5Left + .apply(Join.leftOuterJoin(join5Right)) + .apply( + MapElements.via( + new SimpleFunction>() { + public KV apply(Row join5Row) { + GenericRecord join5Record = + AvroUtils.toGenericRecord(join5Row, JOIN5_SCHEMA); + String key = + BeamExecUtil.buildStringKey(join5Record, "intCol", "stringCol"); + return KV.of(key, join5Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(JOIN5_SCHEMA))) + .apply(Window.into(new GlobalWindows())); + join5.apply(KafkaIOGenericRecord.write().withTopic("ss_join_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/ProjectTestApplication.java b/coral-beam/src/test/beamGeneratedCode/ProjectTestApplication.java new file mode 100644 index 000000000..006395b97 --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/ProjectTestApplication.java @@ -0,0 +1,150 @@ +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.excution.BeamAPIUtil; +import com.linkedin.beam.excution.BeamExecUtil; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import java.util.Objects; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.calcite.piglet.PigRelSqlUdfs; +import org.apache.pig.builtin.ENDSWITH; +import org.apache.pig.data.Tuple; + +public class ProjectTestApplication { + private static final Schema PROJECT2_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"intStringCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"longStringCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"doubleStringCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"stringStringCol\",\"type\":[\"string\",\"null\"]},{\"name\":\"constStringString\",\"type\":\"string\"},{\"name\":\"constLongString\",\"type\":[\"string\",\"null\"]},{\"name\":\"intLongCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"longLongCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"doubleLongCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"stringLongCol\",\"type\":[\"long\",\"null\"]},{\"name\":\"constLongLong\",\"type\":\"long\"},{\"name\":\"constStringLong\",\"type\":[\"long\",\"null\"]},{\"name\":\"intIntCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"longIntCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"doubleIntCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringIntCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"constIntInt\",\"type\":\"int\"},{\"name\":\"constStringInt\",\"type\":[\"int\",\"null\"]},{\"name\":\"intDoubleCol\",\"type\":[\"double\",\"null\"]},{\"name\":\"longDoubleCol\",\"type\":[\"double\",\"null\"]},{\"name\":\"doubleDoubleCol\",\"type\":[\"double\",\"null\"]},{\"name\":\"stringDoubleCol\",\"type\":[\"double\",\"null\"]},{\"name\":\"intConstDoubleCol\",\"type\":\"double\"},{\"name\":\"stringContsDoubleCol\",\"type\":[\"double\",\"null\"]},{\"name\":\"recordCol\",\"type\":{\"type\":\"record\",\"name\":\"record0_recordCol\",\"fields\":[{\"name\":\"_0\",\"type\":\"int\"},{\"name\":\"_1\",\"type\":\"string\"}]}},{\"name\":\"bagCol\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"record0_bagCol\",\"fields\":[{\"name\":\"_0\",\"type\":\"int\"},{\"name\":\"_1\",\"type\":\"string\"}]}}},{\"name\":\"udfCol\",\"type\":[\"boolean\",\"null\"]}]}"); + private static final Schema PROJECT2_COLUMN25_SCHEMA_CONST_11_COMP = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"_0\",\"type\":\"int\"},{\"name\":\"_1\",\"type\":\"string\"}]}"); + private static final Schema PROJECT2_COLUMN24_SCHEMA_CONST_10 = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"_0\",\"type\":\"int\"},{\"name\":\"_1\",\"type\":\"string\"}]}"); + private static final Schema PROJECT2_COLUMN25_SCHEMA_CONST_11 = + new Schema.Parser() + .parse( + "{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"_0\",\"type\":\"int\"},{\"name\":\"_1\",\"type\":\"string\"}]}}"); + private static boolean staticVarInit = false; + private static ENDSWITH org_apache_pig_builtin_ENDSWITH; + + public static void main(String[] args) { + if (!staticVarInit) { + org_apache_pig_builtin_ENDSWITH = new ENDSWITH(); + staticVarInit = true; + } + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read projectStream", + KafkaIOGenericRecord.read() + .withTopic("projectStream") + .withTimestampFn(BeamAPIUtil.withTimeFunction(ImmutableList.of("longCol"))) + .withoutMetadata()); + PCollection> project2 = + stream1 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream1KV) { + final GenericRecord stream1Record = stream1KV.getValue(); + GenericRecord project2Record = new GenericData.Record(PROJECT2_SCHEMA); + final Integer var0 = (Integer) stream1Record.get("intCol"); + final String var1 = Objects.toString(var0, null); + project2Record.put("intStringCol", var1); + final Long var2 = (Long) stream1Record.get("longCol"); + final String var3 = Objects.toString(var2, null); + project2Record.put("longStringCol", var3); + final Double var4 = (Double) stream1Record.get("doubleCol"); + final String var5 = Objects.toString(var4, null); + project2Record.put("doubleStringCol", var5); + final String var6 = Objects.toString(stream1Record.get("stringCol"), null); + project2Record.put("stringStringCol", var6); + project2Record.put("constStringString", "abc"); + final String var7 = String.valueOf(1L); + project2Record.put("constLongString", var7); + final Integer var8 = (Integer) stream1Record.get("intCol"); + final Long var9 = var8 != null ? var8.longValue() : (Long) null; + project2Record.put("intLongCol", var9); + final Long var10 = (Long) stream1Record.get("longCol"); + final Long var11 = var10 != null ? var10 + 2L : (Long) null; + project2Record.put("longLongCol", var11); + final Double var12 = (Double) stream1Record.get("doubleCol"); + final Long var13 = var12 != null ? var12.longValue() : (Long) null; + project2Record.put("doubleLongCol", var13); + final String var14 = Objects.toString(stream1Record.get("stringCol"), null); + final Long var15 = Long.valueOf(var14); + project2Record.put("stringLongCol", var15); + project2Record.put("constLongLong", 3L); + final Long var16 = Long.valueOf("4"); + project2Record.put("constStringLong", var16); + final Integer var17 = (Integer) stream1Record.get("intCol"); + final Integer var18 = var17 != null ? var17 + 5 : (Integer) null; + project2Record.put("intIntCol", var18); + final Long var19 = (Long) stream1Record.get("longCol"); + final Integer var20 = var19 != null ? var19.intValue() : (Integer) null; + project2Record.put("longIntCol", var20); + final Double var21 = (Double) stream1Record.get("doubleCol"); + final Integer var22 = var21 != null ? var21.intValue() : (Integer) null; + project2Record.put("doubleIntCol", var22); + final String var23 = Objects.toString(stream1Record.get("stringCol"), null); + final Integer var24 = Integer.valueOf(var23); + project2Record.put("stringIntCol", var24); + project2Record.put("constIntInt", 6); + final Integer var25 = Integer.valueOf("7"); + project2Record.put("constStringInt", var25); + final Integer var26 = (Integer) stream1Record.get("intCol"); + final Double var27 = var26 != null ? var26.doubleValue() : (Double) null; + project2Record.put("intDoubleCol", var27); + final Long var28 = (Long) stream1Record.get("longCol"); + final Double var29 = var28 != null ? var28.doubleValue() : (Double) null; + project2Record.put("longDoubleCol", var29); + final Double var30 = (Double) stream1Record.get("doubleCol"); + project2Record.put("doubleDoubleCol", var30); + final String var31 = Objects.toString(stream1Record.get("stringCol"), null); + final Double var32 = Double.valueOf(var31); + project2Record.put("stringDoubleCol", var32); + project2Record.put("intConstDoubleCol", 9.0D); + final Double var33 = Double.valueOf("10.5"); + project2Record.put("stringContsDoubleCol", var33); + project2Record.put( + "recordCol", + BeamExecUtil.toAvroRecord( + PigRelSqlUdfs.buildTuple(1, "a"), + PROJECT2_COLUMN24_SCHEMA_CONST_10)); + project2Record.put( + "bagCol", + BeamExecUtil.toAvroArray( + PigRelSqlUdfs.buildBag( + BeamExecUtil.toAvroRecord( + PigRelSqlUdfs.buildTuple(1, "a"), + PROJECT2_COLUMN25_SCHEMA_CONST_11_COMP), + BeamExecUtil.toAvroRecord( + PigRelSqlUdfs.buildTuple(2, "b"), + PROJECT2_COLUMN25_SCHEMA_CONST_11_COMP)), + PROJECT2_COLUMN25_SCHEMA_CONST_11)); + final String var34 = Objects.toString(stream1Record.get("stringCol"), null); + final Tuple var35 = PigRelSqlUdfs.buildTuple(var34, "foo"); + final Boolean var36 = org_apache_pig_builtin_ENDSWITH.exec(var35); + project2Record.put("udfCol", var36); + return KV.of(stream1KV.getKey(), project2Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT2_SCHEMA))); + project2.apply(KafkaIOGenericRecord.write().withTopic("project_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/ScanTestApplication.java b/coral-beam/src/test/beamGeneratedCode/ScanTestApplication.java new file mode 100644 index 000000000..eb2b49267 --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/ScanTestApplication.java @@ -0,0 +1,25 @@ +import com.google.common.collect.ImmutableList; +import com.linkedin.beam.excution.BeamAPIUtil; +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +public class ScanTestApplication { + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read testStream", + KafkaIOGenericRecord.read() + .withTopic("testStream") + .withTimestampFn(BeamAPIUtil.withTimeFunction(ImmutableList.of("longCol"))) + .withoutMetadata()); + stream1.apply(KafkaIOGenericRecord.write().withTopic("scan_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/beamGeneratedCode/UnionTestApplication.java b/coral-beam/src/test/beamGeneratedCode/UnionTestApplication.java new file mode 100644 index 000000000..f07aab1c0 --- /dev/null +++ b/coral-beam/src/test/beamGeneratedCode/UnionTestApplication.java @@ -0,0 +1,103 @@ +import com.linkedin.beam.excution.KafkaIOGenericRecord; +import java.util.Objects; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; + +public class UnionTestApplication { + private static final Schema PROJECT2_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"intCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringCol\",\"type\":[\"string\",\"null\"]}]}"); + private static final Schema PROJECT4_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"intCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringCol\",\"type\":[\"string\",\"null\"]}]}"); + private static final Schema PROJECT6_SCHEMA = + new Schema.Parser() + .parse( + "{\"type\":\"record\",\"name\":\"record0\",\"namespace\":\"rel_avro\",\"fields\":[{\"name\":\"intCol\",\"type\":[\"int\",\"null\"]},{\"name\":\"stringCol\",\"type\":[\"string\",\"null\"]}]}"); + + public static void main(String[] args) { + final PipelineOptions pipelineOpts = PipelineOptionsFactory.create(); + final Pipeline pipeline = Pipeline.create(pipelineOpts); + PCollection> stream1 = + pipeline.apply( + "Read unionStream", + KafkaIOGenericRecord.read().withTopic("unionStream").withoutMetadata()); + PCollection> project2 = + stream1 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream1KV) { + final GenericRecord stream1Record = stream1KV.getValue(); + GenericRecord project2Record = new GenericData.Record(PROJECT2_SCHEMA); + final Integer var0 = (Integer) stream1Record.get("intCol"); + project2Record.put("intCol", var0); + final String var1 = Objects.toString(stream1Record.get("stringCol"), null); + project2Record.put("stringCol", var1); + return KV.of(stream1KV.getKey(), project2Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT2_SCHEMA))); + PCollection> stream3 = + pipeline.apply( + "Read unionStream2", + KafkaIOGenericRecord.read().withTopic("unionStream2").withoutMetadata()); + PCollection> project4 = + stream3 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream3KV) { + final GenericRecord stream3Record = stream3KV.getValue(); + GenericRecord project4Record = new GenericData.Record(PROJECT4_SCHEMA); + final Integer var0 = (Integer) stream3Record.get("intCol"); + final Integer var1 = var0 != null ? var0 + 1 : (Integer) null; + project4Record.put("intCol", var1); + final String var2 = Objects.toString(stream3Record.get("stringCol"), null); + project4Record.put("stringCol", var2); + return KV.of(stream3KV.getKey(), project4Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT4_SCHEMA))); + PCollection> stream5 = + pipeline.apply( + "Read unionStream3", + KafkaIOGenericRecord.read().withTopic("unionStream3").withoutMetadata()); + PCollection> project6 = + stream5 + .apply( + MapElements.via( + new SimpleFunction, KV>() { + public KV apply(KV stream5KV) { + final GenericRecord stream5Record = stream5KV.getValue(); + GenericRecord project6Record = new GenericData.Record(PROJECT6_SCHEMA); + final Integer var0 = (Integer) stream5Record.get("intCol"); + final Integer var1 = var0 != null ? var0 + 2 : (Integer) null; + project6Record.put("intCol", var1); + final String var2 = Objects.toString(stream5Record.get("stringCol"), null); + project6Record.put("stringCol", var2); + return KV.of(stream5KV.getKey(), project6Record); + } + })) + .setCoder(KvCoder.of(StringUtf8Coder.of(), AvroCoder.of(PROJECT6_SCHEMA))); + PCollection> union7 = + PCollectionList.of(project2).and(project4).and(project6).apply(Flatten.pCollections()); + union7.apply(KafkaIOGenericRecord.write().withTopic("union_output")); + pipeline.run(); + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/CodegenTestBase.java b/coral-beam/src/test/java/com/linkedin/beam/planner/CodegenTestBase.java new file mode 100644 index 000000000..cf58d5207 --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/CodegenTestBase.java @@ -0,0 +1,61 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import javax.tools.JavaCompiler; +import javax.tools.ToolProvider; +import org.apache.commons.io.FileUtils; +import org.testng.Assert; + + +public class CodegenTestBase { + private static final String ROOT_DIR = String.join(File.separator, System.getProperty("user.dir"), "src/test"); + private static final String BEAM_CODE_DIR = String.join(File.separator, ROOT_DIR, "beamGeneratedCode"); + private static final String TIMESTAMP_FIELD = "timestamp"; + private static final boolean TEST_RESET = false; + + public static CalciteBeamConfig makeBeamConfig(String appName, String outputName) { + return new CalciteBeamConfig(appName, TIMESTAMP_FIELD, outputName); + } + + public static void testBeamCodeGen(CalciteBeamConfig config, String pigQuery) throws Exception { + String javaFileName = config.applicationName + ".java"; + String generatedCode = new BeamCodeGenerator(config).generateJavaCodeFromPigQuery(pigQuery); + validateCode(javaFileName, generatedCode); + } + + private static void validateCode(String javaFileName, String generatedCode) throws Exception { + compileSource(generatedCode, javaFileName); + String expectedCodeFile = String.join(File.separator, BEAM_CODE_DIR, javaFileName); + checkResult(expectedCodeFile, generatedCode); + } + + private static void compileSource(String source, String javaFileName) throws IOException { + String tmpProperty = System.getProperty("java.io.tmpdir"); + Path javaFile = Paths.get(tmpProperty, javaFileName); + Files.write(javaFile, source.getBytes(StandardCharsets.UTF_8)); + JavaCompiler compiler = ToolProvider.getSystemJavaCompiler(); + ByteArrayOutputStream errorStream = new ByteArrayOutputStream(); + int status = compiler.run(null, null, errorStream, javaFile.toFile().getAbsolutePath()); + String errorMessage = errorStream.toString(); + Assert.assertEquals(0, status, errorMessage); + } + + private static void checkResult(String expectedFile, String code) throws IOException { + if (TEST_RESET) { + FileUtils.write(new File(expectedFile), code); + } else { + Assert.assertEquals(code, FileUtils.readFileToString(new File(expectedFile))); + } + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/TestAggregate.java b/coral-beam/src/test/java/com/linkedin/beam/planner/TestAggregate.java new file mode 100644 index 000000000..139661c52 --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/TestAggregate.java @@ -0,0 +1,53 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import org.testng.annotations.Test; + +import static com.linkedin.beam.planner.CodegenTestBase.*; + + +public class TestAggregate { + + @Test + public void testDistinct() throws Exception { + CalciteBeamConfig config = makeBeamConfig("DistinctTestApplication", "distinct_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("testStream")); + String script = "" + + "inputData = load 'dummy' as (intCol:int, doubleCol:double, stringCol:chararray);\n" + + "aggregate = distinct inputData;\n" + + "store aggregate into 'output';"; + testBeamCodeGen(config, script); + } + + @Test + public void testAggregate() throws Exception { + CalciteBeamConfig config = makeBeamConfig("AggregateTestApplication", "aggregate_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("testStream")); + String script = "" + + "inputData = load 'dummy' as (strGroupCol:chararray, longGroupCol:long, intCol:int, longCol:long, doubleCol:double, stringCol:chararray);\n" + + "group1 = group inputData by (strGroupCol, longGroupCol);\n" + + "aggregate1 = foreach group1 generate\n" + + " group.strGroupCol,\n" + + " group.longGroupCol,\n" + + " MAX(inputData.longCol) as maxLong,\n" + + " MAX(inputData.intCol) as maxInt,\n" + + " MAX(inputData.doubleCol) as maxDouble,\n" + + " MAX(inputData.stringCol) as maxString,\n" + + " MIN(inputData.longCol) as minLong,\n" + + " MIN(inputData.intCol) as minInt,\n" + + " MIN(inputData.doubleCol) as minDouble,\n" + + " MIN(inputData.stringCol) as minString,\n" + + " SUM(inputData.longCol) as sumLong,\n" + + " SUM(inputData.intCol) as sumInt,\n" + + " SUM(inputData.doubleCol) as sumDouble,\n" + + " COUNT(inputData.longCol) as countLong,\n" + + " COUNT(inputData) as countStar,\n" + + " inputData.longCol as longMultiset;\n" + + "store aggregate1 into 'output';"; + testBeamCodeGen(config, script); + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/TestArrayFlatten.java b/coral-beam/src/test/java/com/linkedin/beam/planner/TestArrayFlatten.java new file mode 100644 index 000000000..c9bfb4ea5 --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/TestArrayFlatten.java @@ -0,0 +1,41 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import org.testng.annotations.Test; + +import static com.linkedin.beam.planner.CodegenTestBase.*; + + +public class TestArrayFlatten { + + @Test + public void testFlattenOneColumn() throws Exception { + CalciteBeamConfig config = makeBeamConfig("ArrayFlatten1CTestApplication", "array_flatten1_output"); + config.tableToInputStreams.put("dummy_array", new CalciteBeamConfig.StreamConfig("arrayFlattenStream")); + String script = "" + + "inputData = load 'dummy_array' as (key:chararray, intCol:int, longArray:{(elem:long)});\n" + + "flatten1 = foreach inputData generate\n" + + " key,\n" + + " flatten(longArray) as longCol;\n" + + "store flatten1 into 'output';"; + testBeamCodeGen(config, script); + } + + @Test + public void testFlattenTwoColumns() throws Exception { + CalciteBeamConfig config = makeBeamConfig("ArrayFlatten2CTestApplication", "array_flatten1_output"); + config.tableToInputStreams.put("dummy_array", new CalciteBeamConfig.StreamConfig("arrayFlattenStream")); + String script = "" + + "inputData = load 'dummy_array' as (key:chararray, intCol:int, longArray:{(lElem:long)}, doubleArray:{(dElem:double)});\n" + + "flatten2 = foreach inputData generate\n" + + " key,\n" + + " flatten(longArray) as longCol,\n" + + " flatten(doubleArray) as doubleCol;\n" + + "store flatten2 into 'output';"; + testBeamCodeGen(config, script); + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/TestFilter.java b/coral-beam/src/test/java/com/linkedin/beam/planner/TestFilter.java new file mode 100644 index 000000000..71b982f5c --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/TestFilter.java @@ -0,0 +1,25 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import org.testng.annotations.Test; + +import static com.linkedin.beam.planner.CodegenTestBase.*; + + +public class TestFilter { + + @Test + public void testFilter() throws Exception { + CalciteBeamConfig config = makeBeamConfig("FilterTestApplication", "filter_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("testStream")); + String script = "" + + "inputData = load 'dummy' as (intCol:int, doubleCol:double, stringCol:chararray);\n" + + "filterData = FILTER inputData BY intCol > 2 AND (doubleCol * doubleCol < 1.8 OR stringCol == 'test');\n" + + "store filterData into 'output';"; + testBeamCodeGen(config, script); + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/TestJoin.java b/coral-beam/src/test/java/com/linkedin/beam/planner/TestJoin.java new file mode 100644 index 000000000..6277800ea --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/TestJoin.java @@ -0,0 +1,29 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import org.testng.annotations.Test; + +import static com.linkedin.beam.planner.CodegenTestBase.*; + + +public class TestJoin { + + @Test + public void testStreamStreamJoin() throws Exception { + CalciteBeamConfig config = makeBeamConfig("JoinTestApplication", "ss_join_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("joinStream")); + config.tableToInputStreams.put("dummy2", new CalciteBeamConfig.StreamConfig("joinStream2")); + String script = "" + + "inputData1 = load 'dummy' as (intCol:int, doubleCol:double, stringCol:chararray);\n" + + "inputData1 = foreach inputData1 generate intCol, stringCol, doubleCol;\n" + + "inputData2 = load 'dummy2' as (intCol:int, longCol:long, stringCol:chararray);\n" + + "inputData2 = foreach inputData2 generate intCol, stringCol, longCol;\n" + + "final = JOIN inputData1 by (intCol, stringCol) LEFT OUTER, inputData2 by (intCol, stringCol);\n" + + "store final into 'output';"; + testBeamCodeGen(config, script); + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/TestProject.java b/coral-beam/src/test/java/com/linkedin/beam/planner/TestProject.java new file mode 100644 index 000000000..d9cee6cd8 --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/TestProject.java @@ -0,0 +1,55 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import com.google.common.collect.ImmutableList; +import org.testng.annotations.Test; + +import static com.linkedin.beam.planner.CodegenTestBase.*; + + +public class TestProject { + + @Test + public void testProject() throws Exception { + CalciteBeamConfig config = makeBeamConfig("ProjectTestApplication", "project_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("projectStream", + ImmutableList.of("longCol"))); + String SCRIPT = + "" + "inputData = load 'dummy' as (intCol:int, longCol:long, doubleCol:double, stringCol:chararray);\n" + + "project = foreach inputData generate\n" + + " (chararray) intCol as intStringCol,\n" + + " (chararray) longCol as longStringCol,\n" + + " (chararray) doubleCol as doubleStringCol,\n" + + " (chararray) stringCol as stringStringCol,\n" + + " (chararray) 'abc' as constStringString,\n" + + " (chararray) 1L as constLongString,\n" + + " (long) intCol as intLongCol,\n" + + " (long) (longCol + 2) as longLongCol,\n" + + " (long) doubleCol as doubleLongCol,\n" + + " (long) stringCol as stringLongCol,\n" + + " (long) 3L as constLongLong,\n" + + " (long) '4' as constStringLong,\n" + + " (int) (intCol + 5) as intIntCol,\n" + + " (int) longCol as longIntCol,\n" + + " (int) doubleCol as doubleIntCol,\n" + + " (int) stringCol as stringIntCol,\n" + + " (int) 6 as constIntInt,\n" + + " (int) '7' as constStringInt,\n" + + " (double) intCol as intDoubleCol,\n" + + " (double) longCol as longDoubleCol,\n" + + " (double) doubleCol as doubleDoubleCol,\n" + + " (double) stringCol as stringDoubleCol,\n" + + " (double) 9 as intConstDoubleCol,\n" + + " (double) '10.5' as stringContsDoubleCol,\n" + + " (1, 'a') as recordCol,\n" + + " {(1, 'a'), (2, 'b')} as bagCol," + + " ENDSWITH(stringCol, 'foo') as udfCol;\n" + + "store project into 'output';"; + + testBeamCodeGen(config, SCRIPT); + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/TestScan.java b/coral-beam/src/test/java/com/linkedin/beam/planner/TestScan.java new file mode 100644 index 000000000..82d5a3322 --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/TestScan.java @@ -0,0 +1,52 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import com.google.common.collect.ImmutableList; +import org.testng.annotations.Test; + +import static com.linkedin.beam.planner.CodegenTestBase.*; +import static org.testng.Assert.*; + + +public class TestScan { + private final String SCRIPT = "" + + "inputData = load 'dummy' as (a:int, longCol:long, stringCol:chararray);\n" + + "store inputData into 'output';"; + + @Test + public void testScan() throws Exception { + CalciteBeamConfig config = makeBeamConfig("ScanTestApplication", "scan_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("testStream", ImmutableList.of("longCol"))); + testBeamCodeGen(config, SCRIPT); + } + + @Test + public void testInvalidTimeSchema() throws Exception { + CalciteBeamConfig config = makeBeamConfig("ScanTestApplication", "scan_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("testStream", + ImmutableList.of("invalidTimeColumn"))); + try { + testBeamCodeGen(config, SCRIPT); + fail("Codegen should fail on invalid time column"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid schema for time field")); + } + } + + @Test + public void testInvalidTimeType() throws Exception { + CalciteBeamConfig config = makeBeamConfig("ScanTestApplication", "scan_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("testStream", + ImmutableList.of("stringCol"))); + try { + testBeamCodeGen(config, SCRIPT); + fail("Codegen should fail on invalid time column"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid type for time column")); + } + } +} diff --git a/coral-beam/src/test/java/com/linkedin/beam/planner/TestUnion.java b/coral-beam/src/test/java/com/linkedin/beam/planner/TestUnion.java new file mode 100644 index 000000000..4817b3da4 --- /dev/null +++ b/coral-beam/src/test/java/com/linkedin/beam/planner/TestUnion.java @@ -0,0 +1,38 @@ +/** + * Copyright 2019 LinkedIn Corporation. All rights reserved. + * Licensed under the BSD-2 Clause license. + * See LICENSE in the project root for license information. + */ +package com.linkedin.beam.planner; + +import org.testng.annotations.Test; + +import static com.linkedin.beam.planner.CodegenTestBase.*; + + +public class TestUnion { + + @Test + public void testUnion() throws Exception { + CalciteBeamConfig config = makeBeamConfig("UnionTestApplication", "union_output"); + config.tableToInputStreams.put("dummy", new CalciteBeamConfig.StreamConfig("unionStream")); + config.tableToInputStreams.put("dummy2", new CalciteBeamConfig.StreamConfig("unionStream2")); + config.tableToInputStreams.put("dummy3", new CalciteBeamConfig.StreamConfig("unionStream3")); + String script = "" + + "inputData = load 'dummy' as (intCol:int, doubleCol:double, stringCol:chararray);\n" + + "child1 = foreach inputData generate\n" + + " intCol as intCol,\n" + + " stringCol as stringCol;\n" + + "inputData2 = load 'dummy2' as (intCol:int, longCol:long, stringCol:chararray);\n" + + "child2 = foreach inputData2 generate\n" + + " intCol + 1 as intCol,\n" + + " stringCol as stringCol;\n" + + "inputData3 = load 'dummy3' as (intCol:int, booleanCol:boolean, stringCol:chararray);\n" + + "child3 = foreach inputData3 generate\n" + + " intCol + 2 as intCol,\n" + + " stringCol as stringCol;\n" + + "final = UNION child1, child2, child3;\n" + + "store final into 'output';"; + testBeamCodeGen(config, script); + } +} diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index dbb676f28..059e8d688 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -11,7 +11,8 @@ def versions = [ 'coral-calcite': '0.0.3', 'hive': '1.2.2', 'hadoop': '2.7.0', - 'pig': '0.15.0' + 'pig': '0.15.0', + 'beam': '2.25.0' ] ext.deps = [ @@ -39,5 +40,9 @@ ext.deps = [ 'pig': [ 'pig': "org.apache.pig:pig:${versions['pig']}:h2", 'pigunit': "org.apache.pig:pigunit:${versions['pig']}" + ], + 'beam': [ + 'java-core': "org.apache.beam:beam-runners-core-java:${versions['beam']}", + 'java-io-kafka': "org.apache.beam:beam-sdks-java-io-kafka:${versions['beam']}" ] ] \ No newline at end of file diff --git a/gradle/shipkit.gradle b/gradle/shipkit.gradle index cdd5831cb..e247cbae1 100644 --- a/gradle/shipkit.gradle +++ b/gradle/shipkit.gradle @@ -9,7 +9,7 @@ shipkit { git.releasableBranchRegex = "master|release/.+" - team.developers = ['wmoustafa:Walaa Eldin Moustafa', 'khaitranq:Khai Tranh', 'funcheetah:Wenye Zhang', 'shardulm94:Shardul Mahadik', 'hotsushi:Sushant Raikar'] + team.developers = ['wmoustafa:Walaa Eldin Moustafa', 'khaitranq:Khai Tran', 'funcheetah:Wenye Zhang', 'shardulm94:Shardul Mahadik', 'hotsushi:Sushant Raikar'] } allprojects { diff --git a/settings.gradle b/settings.gradle index aa18a5555..a014f6ede 100644 --- a/settings.gradle +++ b/settings.gradle @@ -1,4 +1,13 @@ -def modules = ['coral-hive', 'coral-pig', 'coral-presto', 'coral-spark','coral-schema','coral-spark-plan'] +def modules = [ + 'coral-beam', + 'coral-beam-runtime', + 'coral-hive', + 'coral-pig', + 'coral-presto', + 'coral-spark', + 'coral-schema', + 'coral-spark-plan' +] modules.each { module -> if (!file(module).directory) {