From b7c8d52da9bc0d2abe23533253d071d2b86e4bef Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Fri, 3 Dec 2021 20:51:07 +0530 Subject: [PATCH 01/24] feat(spark-lineage): add ability to push data lineage from spark to datahub --- .gitignore | 5 + build.gradle | 8 +- spark-lineage/README.md | 94 ++++ spark-lineage/bin/.gitignore | 2 + spark-lineage/build.gradle | 138 +++++ .../lineage/consumer/impl/MCPEmitter.java | 67 +++ .../lineage/consumer/impl/RESTEmitter.java | 85 +++ .../interceptor/DatahubLineageEmitter.java | 286 ++++++++++ .../spark/interceptor/DatasetExtractor.java | 156 ++++++ .../spark/interceptor/LineageUtils.java | 125 +++++ .../lineage/spark/model/AppEndEvent.java | 46 ++ .../lineage/spark/model/AppStartEvent.java | 52 ++ .../lineage/spark/model/DatasetLineage.java | 35 ++ .../lineage/spark/model/LineageConsumer.java | 6 + .../lineage/spark/model/LineageEvent.java | 22 + .../spark/model/SQLQueryExecEndEvent.java | 47 ++ .../spark/model/SQLQueryExecStartEvent.java | 119 ++++ .../model/dataset/CatalogTableDataset.java | 29 + .../spark/model/dataset/HdfsPathDataset.java | 31 ++ .../spark/model/dataset/JdbcDataset.java | 39 ++ .../spark/model/dataset/SparkDataset.java | 7 + .../datahub/lineage/TestSparkJobsLineage.java | 522 ++++++++++++++++++ .../src/test/resources/data/in1.csv/part1.csv | 3 + .../src/test/resources/data/in2.csv/part1.csv | 4 + .../resources/expected/testHdfsInHiveOut.json | 3 + .../resources/expected/testHdfsInJdbcOut.json | 1 + .../resources/expected/testHdfsInOut.json | 1 + .../expected/testHdfsJdbcInJdbcOut.json | 1 + .../testHdfsJdbcInJdbcOutTwoLevel.json | 1 + .../resources/expected/testHiveInHiveOut.json | 4 + .../apache/spark/log4j-defaults.properties | 8 + 31 files changed, 1946 insertions(+), 1 deletion(-) create mode 100644 spark-lineage/README.md create mode 100644 spark-lineage/bin/.gitignore create mode 100644 spark-lineage/build.gradle create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatasetExtractor.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppEndEvent.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppStartEvent.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/DatasetLineage.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageConsumer.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageEvent.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecEndEvent.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecStartEvent.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/CatalogTableDataset.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/HdfsPathDataset.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/JdbcDataset.java create mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/SparkDataset.java create mode 100644 spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java create mode 100644 spark-lineage/src/test/resources/data/in1.csv/part1.csv create mode 100644 spark-lineage/src/test/resources/data/in2.csv/part1.csv create mode 100644 spark-lineage/src/test/resources/expected/testHdfsInHiveOut.json create mode 100644 spark-lineage/src/test/resources/expected/testHdfsInJdbcOut.json create mode 100644 spark-lineage/src/test/resources/expected/testHdfsInOut.json create mode 100644 spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOut.json create mode 100644 spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOutTwoLevel.json create mode 100644 spark-lineage/src/test/resources/expected/testHiveInHiveOut.json create mode 100644 spark-lineage/src/test/resources/org/apache/spark/log4j-defaults.properties diff --git a/.gitignore b/.gitignore index 1b691ad0aaca41..c93e3efe19df25 100644 --- a/.gitignore +++ b/.gitignore @@ -39,4 +39,9 @@ MANIFEST # Mac OS **/.DS_Store +#spark-lineage +**/spark-lineage/metastore_db/ +**/spark-lineage/**/derby.log +**/spark-lineage/**/hive/ +**/spark-lineage/**/out.csv/ .vscode diff --git a/build.gradle b/build.gradle index 77c30b3680b427..dc9ca7e85a0509 100644 --- a/build.gradle +++ b/build.gradle @@ -3,7 +3,6 @@ buildscript { ext.gmaVersion = '0.2.81' ext.pegasusVersion = '28.3.7' ext.mavenVersion = '3.6.3' - apply from: './repositories.gradle' buildscript.repositories.addAll(project.repositories) dependencies { @@ -11,6 +10,7 @@ buildscript { classpath 'com.github.node-gradle:gradle-node-plugin:2.2.4' classpath 'com.commercehub.gradle.plugin:gradle-avro-plugin:0.8.1' classpath 'org.springframework.boot:spring-boot-gradle-plugin:2.1.4.RELEASE' + classpath 'com.github.jengelman.gradle.plugins:shadow:5.2.0' } } @@ -68,6 +68,8 @@ project.ext.externalDependency = [ 'guava': 'com.google.guava:guava:27.0.1-jre', 'h2': 'com.h2database:h2:1.4.196', 'hadoopClient': 'org.apache.hadoop:hadoop-client:3.1.1', + 'hadoopCommon':'org.apache.hadoop:hadoop-common:2.7.2', + 'hadoopMapreduceClient':'org.apache.hadoop:hadoop-mapreduce-client-core:2.7.2', 'hibernateCore': 'org.hibernate:hibernate-core:5.2.16.Final', 'httpClient': 'org.apache.httpcomponents:httpclient:4.5.9', 'iStackCommons': 'com.sun.istack:istack-commons-runtime:4.0.1', @@ -114,6 +116,8 @@ project.ext.externalDependency = [ 'rythmEngine': 'org.rythmengine:rythm-engine:1.3.0', 'servletApi': 'javax.servlet:javax.servlet-api:3.1.0', 'shiroCore': 'org.apache.shiro:shiro-core:1.7.1', + 'sparkSql' : 'org.apache.spark:spark-sql_2.11:2.4.8', + 'sparkHive' : 'org.apache.spark:spark-hive_2.11:2.4.8', 'springBeans': 'org.springframework:spring-beans:5.2.3.RELEASE', 'springContext': 'org.springframework:spring-context:5.2.3.RELEASE', 'springCore': 'org.springframework:spring-core:5.2.3.RELEASE', @@ -129,7 +133,9 @@ project.ext.externalDependency = [ 'testng': 'org.testng:testng:7.3.0', 'testContainers': 'org.testcontainers:testcontainers:1.15.1', 'testContainersJunit': 'org.testcontainers:junit-jupiter:1.15.1', + 'testContainersPostgresql':'org.testcontainers:postgresql:1.2.0', 'testContainersElasticsearch': 'org.testcontainers:elasticsearch:1.15.3', + 'wiremock':'com.github.tomakehurst:wiremock:2.10.0', 'zookeeper': 'org.apache.zookeeper:zookeeper:3.4.14' ] diff --git a/spark-lineage/README.md b/spark-lineage/README.md new file mode 100644 index 00000000000000..0a640068d21540 --- /dev/null +++ b/spark-lineage/README.md @@ -0,0 +1,94 @@ +# Metadata Ingestion: Emitters + +Emitters are datahub artifacts that you can plug into your source data platforms. Depending on the capabilities provided by the data platform, an emitter can extract metadata such as datasets, jobs, lineage etc. whenever relevant events occur. The emitter then pushes this metadata to Datahub's GMS using Kafka and/or REST. + +All emitters thus typically expect a minimal configuration to specify the connection details to GMS. + +## Spark lineage emitter +The Spark lineage emitter is a java library that provides a Spark listener implementation "DatahubLineageEmitter". The DatahubLineageEmitter listens to events such application start/end, and SQLExecution start/end to create pipelines (i.e. DataJob) and tasks (i.e. DataFlow) in Datahub along with lineage. + +### Configuring Spark emitter +Listener configuration can be done using a config file or while creating a spark Session. + +#### Config file for spark-submit +When running jobs using spark-submit, the listener is to be configured in the config file. + +``` +spark.master spark://spark-master:7077 + +#Configuring datahub spark listner jar +spark.jars.packages io.acryl:spark-lineage:0.0.1 +spark.extraListeners com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter +spark.datahub.lineage.mcpEmitter.gmsUrl http://localhost:8080 +``` + +#### Configuring with SparkSession Builder for notebooks +When running interactive jobs from a notebook, the listener can be configured while building the Spark Session. + +```python +spark = SparkSession.builder \ + .master("spark://spark-master:7077") \ + .appName("test-application") \ + .config("spark.jars.packages","io.acryl:spark-lineage:0.0.1") \ + .config("spark.extraListeners","com.linkedin.datahub.lineage.interceptor.spark.DatahubLineageEmitter") \ + .config("spark.datahub.lineage.mcpEmitter.gmsUrl", "http://localhost:8080") \ + .enableHiveSupport() \ + .getOrCreate() +``` + +### Model mapping +A pipeline is created per Spark . +A task is created per unique Spark query execution within an app. + +#### Custom properties & relating to Spark UI +The following custom properties in pipelines and tasks relate to the Spark UI: +- appName and appId in a pipeline can be used to determine the Spark application +- description and SQLQueryId in a task can be used to determine the Query Execution within the application on the SQL tab of Spark UI + +Other custom properties of pipelines and tasks capture the start and end times of execution etc. +The query plan is captured in the *queryPlan* property of a task. + +### Release notes for v0.0.1 +In this version, basic dataset-level lineage is captured using the model mapping as mentioned earlier. + +#### Spark versions supported +The primary version tested is Spark/Scala version 2.4.8/2_11. +We anticipate this to work well with other Spark 2.4.x versions and Scala 2_11. + +Support for other Spark versions is planned in the very near future. + +#### Environments tested with +This initial release has been tested with the following environments: +- spark-submit of Python/Java applications to local and remote servers +- notebooks + +Note that testing for other environments such as Databricks and standalone applications is planned in near future. + +#### Spark commands supported +Below is a list of Spark commands that are parsed currently: +- InsertIntoHadoopFsRelationCommand +- SaveIntoDataSourceCommand (jdbc) +- CreateHiveTableAsSelectCommand +- InsertIntoHiveTable + +Effectively, these support data sources/sinks corresponding to Hive, HDFS and JDBC. + +#### Spark commands not yet supported +- View related commands +- Cache commands and implications on lineage +- RDD jobs + +#### Important notes on usage + +- It is advisable to ensure appName is used appropriately to ensure you can trace lineage from a pipeline back to your source code. + +- If multiple apps with the same appName run concurrently, dataset-lineage will be captured correctly but the custom-properties e.g. app-id, SQLQueryId would be unreliable. We expect this to be quite rare. + +- If spark execution fails, then an empty pipeline would still get created, but it may not have any tasks. + +- For HDFS sources, the folder (name) is regarded as the dataset (name) to align with typical storage of parquet/csv formats. + +### Known limitations +- Only postgres supported for JDBC sources in this initial release. Support for other driver URL formats will be added in future. +- Behavior with cached datasets is not fully specified/defined in context of lineage. +- There is a possibility that very short-lived jobs that run within a few milliseconds may not be captured by the listener. This should not cause an issue for realistic Spark applications. diff --git a/spark-lineage/bin/.gitignore b/spark-lineage/bin/.gitignore new file mode 100644 index 00000000000000..7eed456bec8db3 --- /dev/null +++ b/spark-lineage/bin/.gitignore @@ -0,0 +1,2 @@ +/main/ +/test/ diff --git a/spark-lineage/build.gradle b/spark-lineage/build.gradle new file mode 100644 index 00000000000000..660969de23ab10 --- /dev/null +++ b/spark-lineage/build.gradle @@ -0,0 +1,138 @@ +apply plugin: 'java' +apply plugin: 'com.github.johnrengelman.shadow' +apply plugin: 'maven' +apply plugin: 'signing' + +dependencies { + + //Needed for tie breaking of guava version need for spark and wiremock + compile(externalDependency.hadoopMapreduceClient) { + force = true + } + + compile(externalDependency.hadoopCommon) { + force = true + } // required for org.apache.hadoop.util.StopWatch + + compile(externalDependency.commonsIo) { + force = true + } // required for org.apache.commons.io.Charsets that is used internally + + compileOnly externalDependency.lombok + annotationProcessor externalDependency.lombok + + implementation(project(':metadata-models')) { + exclude group: "org.antlr" + exclude group: "com.google.guava" // causes issues with Guava Stopwatch constructor + } + + implementation(externalDependency.sparkSql){ + exclude group: "org.apache.hadoop" + } + implementation(externalDependency.sparkHive){ + exclude group: "org.apache.hadoop" + } + + testImplementation(externalDependency.postgresql) + + testImplementation externalDependency.mockito + + testImplementation(externalDependency.wiremock){ + exclude group: "com.fasterxml.jackson.core" + } // older version to allow older guava + + testImplementation(externalDependency.testContainersPostgresql) // older version to allow older jackson +} + + + +shadowJar { + zip64=true + classifier='' + dependencies { + exclude(dependency("org.apache.hadoop::")) + exclude(dependency("org.apache.spark::")) + exclude(dependency(externalDependency.commonsIo)) + } +} + + + +test { + useJUnit() +} + +assemble { + dependsOn shadowJar +} + +task sourceJar(type: Jar) { + classifier 'sources' + from sourceSets.main.allJava +} + +task javadocJar(type: Jar, dependsOn: javadoc) { + classifier 'javadoc' + from javadoc.destinationDir +} + +artifacts { + archives shadowJar +} + +// uploadArchives { +// repositories { +// mavenDeployer { +// def ossrhUsername = System.getenv('RELEASE_USERNAME') +// def ossrhPassword = System.getenv('RELEASE_PASSWORD') +// beforeDeployment { MavenDeployment deployment -> signing.signPom(deployment) } + +// repository(url: "https://s01.oss.sonatype.org/service/local/staging/deploy/maven2/") { +// authentication(userName: ossrhUsername, password: ossrhPassword) +// } + +// snapshotRepository(url: "https://s01.oss.sonatype.org/content/repositories/snapshots/") { +// authentication(userName: ossrhUsername, password: ossrhPassword) +// } + +// pom.project { +// //No need to specify name here. Name is always picked up from project name +// //name 'spark-lineage' +// packaging 'jar' +// // optionally artifactId can be defined here +// description 'Library to push data lineage from spark to datahub' +// url 'https://datahubproject.io' + +// scm { +// connection 'scm:git:git://github.com/linkedin/datahub.git' +// developerConnection 'scm:git:ssh://github.com:linkedin/datahub.git' +// url 'https://github.com/linkedin/datahub.git' +// } + +// licenses { +// license { +// name 'The Apache License, Version 2.0' +// url 'http://www.apache.org/licenses/LICENSE-2.0.txt' +// } +// } + +// developers { +// developer { +// id 'datahub' +// name 'datahub' +// +// } +// } +// } +// } +// } +// } + + +// signing { +// def signingKey = findProperty("signingKey") +// def signingPassword = findProperty("signingPassword") +// useInMemoryPgpKeys(signingKey, signingPassword) +// sign configurations.archives +// } + diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java new file mode 100644 index 00000000000000..017d75c6078375 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java @@ -0,0 +1,67 @@ +package com.linkedin.datahub.lineage.consumer.impl; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkEnv; + +import com.linkedin.datahub.lineage.spark.model.LineageConsumer; +import com.linkedin.datahub.lineage.spark.model.LineageEvent; +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class MCPEmitter implements LineageConsumer { + + private static final String GMS_URL_KEY = "spark.datahub.lineage.mcpEmitter.gmsUrl"; + private static final String SENTINEL = "moot"; + + private ConcurrentHashMap singleton = new ConcurrentHashMap<>(); + + private void emit(List mcps) { + RESTEmitter emitter = emitter(); + if (emitter != null) { + mcps.forEach(mcp -> { + log.debug("Emitting \n" + mcp); + try { + emitter.emit(mcp); + } catch (IOException e) { + // log error, but don't impact thread + StringWriter s = new StringWriter(); + PrintWriter p = new PrintWriter(s); + e.printStackTrace(p); + log.error(s.toString()); + p.close(); + } + }); + } + } + + // TODO ideally the impl here should not be tied to Spark; the LineageConsumer + // API needs tweaking to include configs + private RESTEmitter emitter() { + singleton.computeIfAbsent(SENTINEL, x -> { + SparkConf conf = SparkEnv.get().conf(); + if (conf.contains(GMS_URL_KEY)) { + String gmsUrl = conf.get(GMS_URL_KEY); + log.debug("REST emitter configured with GMS url " + gmsUrl); + return RESTEmitter.create(gmsUrl); + } + + log.error("GMS URL not configured."); + return null; + }); + + return singleton.get(SENTINEL); + } + + @Override + public void accept(LineageEvent evt) { + emit(evt.toMcps()); + } +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java new file mode 100644 index 00000000000000..6fd3ffee83e3b8 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java @@ -0,0 +1,85 @@ +package com.linkedin.datahub.lineage.consumer.impl; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.HashMap; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.data.template.JacksonDataTemplateCodec; +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@RequiredArgsConstructor +public class RESTEmitter { + + private static final JacksonDataTemplateCodec DATA_TEMPLATE_CODEC = new JacksonDataTemplateCodec(); + + @Getter + private final String gmsUrl; + + public void emit(MetadataChangeProposal mcp) throws IOException { + String payloadJson = DATA_TEMPLATE_CODEC.mapToString(mcp.data()); + ObjectMapper om = new ObjectMapper(); + TypeReference> typeRef = new TypeReference>() { + }; + HashMap o = om.readValue(payloadJson, typeRef); + while (o.values().remove(null)) { + + } + + payloadJson = om.writeValueAsString(o); + payloadJson = "{" + " \"proposal\" :" + payloadJson + "}"; + log.debug("Emitting payload: " + payloadJson + "\n to URL " + this.gmsUrl + "/aspects?action=ingestProposal"); + RESTEmitter.makeRequest(this.gmsUrl + "/aspects?action=ingestProposal", "POST", payloadJson); + } + + public static boolean makeRequest(String urlStr, String method, String payloadJson) throws IOException { + URL url = new URL(urlStr); + HttpURLConnection con = (HttpURLConnection) url.openConnection(); + con.setRequestMethod(method); + con.setRequestProperty("Content-Type", "application/json"); + con.setRequestProperty("X-RestLi-Protocol-Version", "2.0.0"); +// con.setRequestProperty("Accept", "application/json"); + con.setDoOutput(true); + if (payloadJson != null) { + try (OutputStream os = con.getOutputStream()) { + byte[] input = payloadJson.getBytes("utf-8"); + os.write(input, 0, input.length); + } + } + try (BufferedReader br = new BufferedReader(new InputStreamReader(con.getInputStream(), "utf-8"))) { + StringBuilder response = new StringBuilder(); + String responseLine = null; + while ((responseLine = br.readLine()) != null) { + response.append(responseLine.trim()); + } + log.debug("URL: " + urlStr + " Response: " + response.toString()); + } + return true; + + } + + public boolean testConnection() { + try { + RESTEmitter.makeRequest(this.gmsUrl + "/config", "GET", null); + return true; + + } catch (IOException e) { + e.printStackTrace(); + return false; + } + } + + public static RESTEmitter create(String gmsUrl) { + return new RESTEmitter(gmsUrl); + } +} \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java new file mode 100644 index 00000000000000..d39d507e5519bb --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java @@ -0,0 +1,286 @@ +package com.linkedin.datahub.lineage.spark.interceptor; + +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.SparkEnv; +import org.apache.spark.scheduler.SparkListener; +import org.apache.spark.scheduler.SparkListenerApplicationEnd; +import org.apache.spark.scheduler.SparkListenerApplicationStart; +import org.apache.spark.scheduler.SparkListenerEvent; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.plans.QueryPlan; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.QueryExecution; +import org.apache.spark.sql.execution.SQLExecution; +import org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd; +import org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart; + +import com.google.common.base.Splitter; +import com.linkedin.datahub.lineage.spark.model.AppEndEvent; +import com.linkedin.datahub.lineage.spark.model.AppStartEvent; +import com.linkedin.datahub.lineage.spark.model.DatasetLineage; +import com.linkedin.datahub.lineage.spark.model.LineageConsumer; +import com.linkedin.datahub.lineage.spark.model.SQLQueryExecEndEvent; +import com.linkedin.datahub.lineage.spark.model.SQLQueryExecStartEvent; +import com.linkedin.datahub.lineage.spark.model.dataset.SparkDataset; + +import lombok.extern.slf4j.Slf4j; +import scala.collection.JavaConversions; +import scala.runtime.AbstractFunction1; +import scala.runtime.AbstractPartialFunction; + +@Slf4j +public class DatahubLineageEmitter extends SparkListener { + + private static final int THREAD_CNT = 10; + public static final String CONSUMER_TYPE_KEY = "spark.datahub.lineage.consumerTypes"; + + private final Map appDetails = new ConcurrentHashMap<>(); + private final Map> appSqlDetails = new ConcurrentHashMap<>(); + private final Map appPoolDetails = new ConcurrentHashMap<>(); + +// private static LineageConsumer loggingConsumer() { +// log.warn("Lineage consumer not specified. Defaulting to LoggingConsumer."); +// return LineageUtils.LOGGING_CONSUMER; +// } + + private class SqlStartTask implements Runnable { + + private SparkListenerSQLExecutionStart sqlStart; + private SparkContext ctx; + private LogicalPlan plan; + + public SqlStartTask(SparkListenerSQLExecutionStart sqlStart, LogicalPlan plan, SparkContext ctx) { + this.sqlStart = sqlStart; + this.plan = plan; + this.ctx = ctx; + } + + @Override + public void run() { + appSqlDetails.get(ctx.appName()).put(sqlStart.executionId(), + new SQLQueryExecStartEvent(ctx.conf().get("spark.master"), ctx.appName(), ctx.applicationId(), + sqlStart.time(), sqlStart.executionId(), null)); + log.debug("PLAN for execution id: " + ctx.appName() + ":" + sqlStart.executionId() + "\n"); + log.debug(plan.toString()); + + DatasetExtractor extractor = new DatasetExtractor(); + Optional outputDS = extractor.asDataset(plan, ctx, true); + if (!outputDS.isPresent()) { + log.debug("Skipping execution as no output dataset present for execution id: " + ctx.appName() + ":" + + sqlStart.executionId()); + return; + } + + DatasetLineage lineage = new DatasetLineage(sqlStart.description(), plan.toString(), outputDS.get()); + Collection> allInners = new ArrayList<>(); + + plan.collect(new AbstractPartialFunction() { + + @Override + public Void apply(LogicalPlan plan) { + log.debug("CHILD " + plan.getClass() + "\n" + plan + "\n-------------\n"); + Optional inputDS = extractor.asDataset(plan, ctx, false); + inputDS.ifPresent(x -> lineage.addSource(x)); + allInners.addAll(JavaConversions.asJavaCollection(plan.innerChildren())); + return null; + } + + @Override + public boolean isDefinedAt(LogicalPlan x) { + return true; + } + }); + + for (QueryPlan qp : allInners) { + if (!(qp instanceof LogicalPlan)) { + continue; + } + LogicalPlan nestedPlan = (LogicalPlan) qp; + + nestedPlan.collect(new AbstractPartialFunction() { + + @Override + public Void apply(LogicalPlan plan) { + log.debug("INNER CHILD " + plan.getClass() + "\n" + plan + "\n-------------\n"); + Optional inputDS = extractor.asDataset(plan, ctx, false); + inputDS.ifPresent( + x -> log.debug("source added for " + ctx.appName() + "/" + sqlStart.executionId() + ": " + x)); + inputDS.ifPresent(x -> lineage.addSource(x)); + return null; + } + + @Override + public boolean isDefinedAt(LogicalPlan x) { + return true; + } + }); + } + + SQLQueryExecStartEvent evt = new SQLQueryExecStartEvent(ctx.conf().get("spark.master"), ctx.appName(), + ctx.applicationId(), + sqlStart.time(), sqlStart.executionId(), lineage); + + appSqlDetails.get(ctx.appName()).put(sqlStart.executionId(), evt); + + consumers().forEach(c -> c.accept(evt)); // TODO parallel stream here? + + log.debug("LINEAGE \n" + lineage + "\n"); + log.debug("Parsed execution id " + ctx.appName() + ":" + sqlStart.executionId()); + + return; + } + + } + + @Override + public void onApplicationStart(SparkListenerApplicationStart applicationStart) { + try { + log.debug("App started: " + applicationStart); + LineageUtils.findSparkCtx().foreach(new AbstractFunction1() { + + @Override + public Void apply(SparkContext sc) { + String appId = applicationStart.appId().isDefined() ? applicationStart.appId().get() : ""; + AppStartEvent evt = new AppStartEvent(LineageUtils.getMaster(sc), applicationStart.appName(), appId, + applicationStart.time(), applicationStart.sparkUser()); + + consumers().forEach(x -> x.accept(evt)); + // TODO keyed by appName; only latest will be considered. Potential + // inconsistencies not mapped. + appDetails.put(applicationStart.appName(), evt); + appSqlDetails.put(applicationStart.appName(), new ConcurrentHashMap<>()); + ExecutorService pool = Executors.newFixedThreadPool(THREAD_CNT); + appPoolDetails.put(applicationStart.appName(), pool); + return null; + } + }); + super.onApplicationStart(applicationStart); + } catch (Exception e) { + // log error, but don't impact thread + StringWriter s = new StringWriter(); + PrintWriter p = new PrintWriter(s); + e.printStackTrace(p); + log.error(s.toString()); + p.close(); + } + } + + @Override + public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { + try { + LineageUtils.findSparkCtx().foreach(new AbstractFunction1() { + + @Override + public Void apply(SparkContext sc) { + log.debug("Application end event received for appId :" + sc.appName()); + AppStartEvent start = appDetails.remove(sc.appName()); + appPoolDetails.remove(sc.appName()).shutdown(); + appSqlDetails.remove(sc.appName()); + if (start == null) { + log.error( + "Application end event received, but start event missing for appId " + sc.applicationId()); + } else { + AppEndEvent evt = new AppEndEvent(LineageUtils.getMaster(sc), sc.appName(), sc.applicationId(), + applicationEnd.time(), start); + + consumers().forEach(x -> x.accept(evt)); + } + return null; + } + }); + super.onApplicationEnd(applicationEnd); + } catch (Exception e) { + // log error, but don't impact thread + StringWriter s = new StringWriter(); + PrintWriter p = new PrintWriter(s); + e.printStackTrace(p); + log.error(s.toString()); + p.close(); + } + } + + @Override + public void onOtherEvent(SparkListenerEvent event) { + try { + if (event instanceof SparkListenerSQLExecutionStart) { + SparkListenerSQLExecutionStart sqlEvt = (SparkListenerSQLExecutionStart) event; + log.debug("SQL Exec start event with id " + sqlEvt.executionId()); + processExecution(sqlEvt); + } else if (event instanceof SparkListenerSQLExecutionEnd) { + SparkListenerSQLExecutionEnd sqlEvt = (SparkListenerSQLExecutionEnd) event; + log.debug("SQL Exec end event with id " + sqlEvt.executionId()); + processExecutionEnd(sqlEvt); + } + } catch (Exception e) { + // log error, but don't impact thread + StringWriter s = new StringWriter(); + PrintWriter p = new PrintWriter(s); + e.printStackTrace(p); + log.error(s.toString()); + p.close(); + } + } + + public void processExecutionEnd(SparkListenerSQLExecutionEnd sqlEnd) { + LineageUtils.findSparkCtx().foreach(new AbstractFunction1() { + + @Override + public Void apply(SparkContext sc) { + SQLQueryExecStartEvent start = appSqlDetails.get(sc.appName()).remove(sqlEnd.executionId()); + if (start == null) { + log.error("Execution end event received, but start event missing for appId/sql exec Id " + sc.applicationId() + + ":" + sqlEnd.executionId()); + } else if (start.getDatasetLineage() != null) { +// JobStatus status = jobEnd.jobResult().equals(org.apache.spark.scheduler.JobSucceeded$.MODULE$) +// ? JobStatus.COMPLETED +// : JobStatus.FAILED; + SQLQueryExecEndEvent evt = new SQLQueryExecEndEvent(LineageUtils.getMaster(sc), sc.appName(), + sc.applicationId(), + sqlEnd.time(), sqlEnd.executionId(), start); + + consumers().forEach(x -> x.accept(evt)); + } + return null; + } + }); + } + + // TODO sqlEvt.details() unused + private void processExecution(SparkListenerSQLExecutionStart sqlStart) { + QueryExecution queryExec = SQLExecution.getQueryExecution(sqlStart.executionId()); + LogicalPlan plan = queryExec.optimizedPlan(); + SparkSession sess = queryExec.sparkSession(); + SparkContext ctx = sess.sparkContext(); + ExecutorService pool = appPoolDetails.get(ctx.appName()); + pool.execute(new SqlStartTask(sqlStart, plan, ctx)); + } + + private static List consumers() { + SparkConf conf = SparkEnv.get().conf(); + if (conf.contains(CONSUMER_TYPE_KEY)) { + String consumerTypes = conf.get(CONSUMER_TYPE_KEY); + + return StreamSupport.stream(Splitter.on(",").trimResults().split(consumerTypes).spliterator(), false) + .map(x -> LineageUtils.getConsumer(x)).filter(x -> x != null).collect(Collectors.toList()); + } else { + return Collections.singletonList(LineageUtils.getConsumer("mcpEmitter")); + } + + } + +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatasetExtractor.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatasetExtractor.java new file mode 100644 index 00000000000000..5b4578588fe57b --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatasetExtractor.java @@ -0,0 +1,156 @@ +package com.linkedin.datahub.lineage.spark.interceptor; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkContext; +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand; +import org.apache.spark.sql.execution.datasources.HadoopFsRelation; +import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand; +import org.apache.spark.sql.execution.datasources.LogicalRelation; +import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions; +import org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation; +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand; +import org.apache.spark.sql.hive.execution.InsertIntoHiveTable; +import org.apache.spark.sql.sources.BaseRelation; + +import com.google.common.collect.ImmutableSet; +import com.linkedin.datahub.lineage.spark.model.dataset.CatalogTableDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.HdfsPathDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.JdbcDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.SparkDataset; + +import scala.Option; +import scala.collection.JavaConversions; + +public class DatasetExtractor { + private static final Map, PlanToDataset> PLAN_TO_DATASET = new HashMap<>(); + private static final Map, RelationToDataset> REL_TO_DATASET = new HashMap<>(); + private static final Set> OUTPUT_CMD = ImmutableSet + .of(InsertIntoHadoopFsRelationCommand.class, SaveIntoDataSourceCommand.class, + CreateDataSourceTableAsSelectCommand.class, CreateHiveTableAsSelectCommand.class, + InsertIntoHiveTable.class); + // TODO InsertIntoHiveDirCommand, InsertIntoDataSourceDirCommand + + private static interface PlanToDataset { + Optional fromPlanNode(LogicalPlan plan, SparkContext ctx); + } + + private static interface RelationToDataset { + Optional fromRelation(BaseRelation rel, SparkContext ctx); + } + + static { + PLAN_TO_DATASET.put(InsertIntoHadoopFsRelationCommand.class, (p, ctx) -> { + InsertIntoHadoopFsRelationCommand cmd = (InsertIntoHadoopFsRelationCommand) p; + if (cmd.catalogTable().isDefined()) { + return Optional.of(new CatalogTableDataset(cmd.catalogTable().get())); + } + return Optional.of(new HdfsPathDataset(cmd.outputPath())); + }); + + PLAN_TO_DATASET.put(LogicalRelation.class, (p, ctx) -> { + BaseRelation baseRel = ((LogicalRelation) p).relation(); + if (!REL_TO_DATASET.containsKey(baseRel.getClass())) { + return Optional.empty(); + } + return REL_TO_DATASET.get(baseRel.getClass()).fromRelation(baseRel, ctx); + }); + + PLAN_TO_DATASET.put(SaveIntoDataSourceCommand.class, (p, ctx) -> { + /* + * BaseRelation relation; if (((SaveIntoDataSourceCommand) p).dataSource() + * instanceof RelationProvider) { RelationProvider relProvider = + * (RelationProvider) ((SaveIntoDataSourceCommand) p).dataSource(); relation = + * relProvider.createRelation(ctx, ((SaveIntoDataSourceCommand) p).options()); } + * else { SchemaRelationProvider relProvider = (SchemaRelationProvider) + * ((SaveIntoDataSourceCommand) p).dataSource(); relation = + * p.createRelation(ctx, ((SaveIntoDataSourceCommand) p).options(), p.schema()); + * } + */ + SaveIntoDataSourceCommand cmd = (SaveIntoDataSourceCommand) p; + + Map options = JavaConversions.mapAsJavaMap(cmd.options()); + String url = options.get("url"); // e.g. jdbc:postgresql://localhost:5432/sparktestdb + if (!url.contains("jdbc")) { + return Optional.empty(); + } + + String tbl = options.get("dbtable"); + return Optional.of(new JdbcDataset(url, tbl)); + }); + + PLAN_TO_DATASET.put(CreateDataSourceTableAsSelectCommand.class, (p, ctx) -> { + CreateDataSourceTableAsSelectCommand cmd = (CreateDataSourceTableAsSelectCommand) p; + // TODO what of cmd.mode() + return Optional.of(new CatalogTableDataset(cmd.table())); + }); + PLAN_TO_DATASET.put(CreateHiveTableAsSelectCommand.class, (p, ctx) -> { + CreateHiveTableAsSelectCommand cmd = (CreateHiveTableAsSelectCommand) p; + return Optional.of(new CatalogTableDataset(cmd.tableDesc())); + }); + PLAN_TO_DATASET.put(InsertIntoHiveTable.class, (p, ctx) -> { + InsertIntoHiveTable cmd = (InsertIntoHiveTable) p; + return Optional.of(new CatalogTableDataset(cmd.table())); + }); + + PLAN_TO_DATASET.put(HiveTableRelation.class, (p, ctx) -> { + HiveTableRelation cmd = (HiveTableRelation) p; + return Optional.of(new CatalogTableDataset(cmd.tableMeta())); + }); + + REL_TO_DATASET.put(HadoopFsRelation.class, (r, ctx) -> { + List res = JavaConversions.asJavaCollection(((HadoopFsRelation) r).location().rootPaths()).stream() + .map(p -> getDirectoryPath(p, ctx.hadoopConfiguration())) + .distinct() + .collect(Collectors.toList()); + + // TODO mapping to URN TBD + return Optional.of(new HdfsPathDataset(res.get(0))); + }); + REL_TO_DATASET.put(JDBCRelation.class, (r, ctx) -> { + JDBCRelation rel = (JDBCRelation) r; + Option tbl = rel.jdbcOptions().parameters().get(JDBCOptions.JDBC_TABLE_NAME()); + if (tbl.isEmpty()) { + return Optional.empty(); + } + + return Optional.of(new JdbcDataset(rel.jdbcOptions().url(), tbl.get())); + }); + } + + Optional asDataset(LogicalPlan logicalPlan, SparkContext ctx, boolean outputNode) { + if (!outputNode && OUTPUT_CMD.contains(logicalPlan.getClass())) { + return Optional.empty(); + } + + if (!PLAN_TO_DATASET.containsKey(logicalPlan.getClass())) { + return Optional.empty(); + } + + return PLAN_TO_DATASET.get(logicalPlan.getClass()).fromPlanNode(logicalPlan, ctx); + } + + private static Path getDirectoryPath(Path p, Configuration hadoopConf) { + try { + if (p.getFileSystem(hadoopConf).getFileStatus(p).isFile()) { + return p.getParent(); + } else { + return p; + } + } catch (IOException e) { + // log.warn("Unable to get file system for path ", e); + return p; + } + } +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java new file mode 100644 index 00000000000000..aab25fc85e2ef5 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java @@ -0,0 +1,125 @@ +package com.linkedin.datahub.lineage.spark.interceptor; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import javax.annotation.Nonnull; + +import org.apache.spark.SparkContext; +import org.apache.spark.SparkContext$; +import org.apache.spark.sql.SparkSession; + +import com.linkedin.common.urn.DataFlowUrn; +import com.linkedin.data.ByteString; +import com.linkedin.data.template.JacksonDataTemplateCodec; +import com.linkedin.data.template.RecordTemplate; +import com.linkedin.datahub.lineage.consumer.impl.MCPEmitter; +import com.linkedin.datahub.lineage.spark.model.LineageConsumer; +import com.linkedin.mxe.GenericAspect; + +import lombok.extern.slf4j.Slf4j; +import scala.Option; +import scala.runtime.AbstractFunction0; +import scala.runtime.AbstractFunction1; + +@Slf4j +public class LineageUtils { + private static final JacksonDataTemplateCodec DATA_TEMPLATE_CODEC = new JacksonDataTemplateCodec(); + + private static Map consumers = new ConcurrentHashMap<>(); + + public static final LineageConsumer LOGGING_CONSUMER = (x -> log.info(x.toString())); + + // hook for replacing paths during testing. Not the cleanest way, TODO improve. + /* This is for generating urn from a hash of the plan */ + // private static Function PATH_REPLACER = (x -> x); + + static { + // system defined consumers + registerConsumer("mcpEmitter", new MCPEmitter()); + } + + private LineageUtils() { + + } + + // overwrites existing consumer entry of same type + public static void registerConsumer(String consumerType, LineageConsumer consumer) { + consumers.put(consumerType, consumer); + } + + public static LineageConsumer getConsumer(String consumerType) { + return consumers.get(consumerType); + } + + public static DataFlowUrn flowUrn(String master, String appName) { + return new DataFlowUrn("spark", appName, master.replaceAll(":", "_").replaceAll("/", "_").replaceAll("[_]+", "_")); + } + + // Taken from GenericAspectUtils + public static GenericAspect serializeAspect(@Nonnull RecordTemplate aspect) { + GenericAspect genericAspect = new GenericAspect(); + + try { + String aspectStr = DATA_TEMPLATE_CODEC.mapToString(aspect.data()); + genericAspect.setValue( + ByteString.unsafeWrap(aspectStr.getBytes(StandardCharsets.UTF_8))); + genericAspect.setContentType("application/json"); + return genericAspect; + } catch (IOException e) { + throw new RuntimeException(e); + } + + } + + public static Option findSparkCtx() { + return SparkSession.getActiveSession() + .map(new AbstractFunction1() { + + @Override + public SparkContext apply(SparkSession sess) { + return sess.sparkContext(); + } + }) + .orElse(new AbstractFunction0>() { + + @Override + public Option apply() { + return SparkContext$.MODULE$.getActive(); + } + }); + } + + public static String getMaster(SparkContext ctx) { + return ctx.conf().get("spark.master"); + } + + /* This is for generating urn from a hash of the plan */ + +/* + public static String scrubPlan(String plan) { + String s = plan.replaceAll("#[0-9]*", ""); + s = s.replaceAll("JdbcRelationProvider@[0-9a-zA-Z]*,", "JdbcRelationProvider,"); + s = s.replaceAll("InMemoryFileIndex@[0-9a-zA-Z]*,", "InMemoryFileIndex,"); + s = s.replaceAll("Created Time:[^\n]+\n", ""); + s = s.replaceAll("Last Access:[^\n]+\n", ""); + s = s.replaceAll("Owner:[^\n]+\n", ""); + s = s.replaceAll("Statistics:[^\n]+\n", ""); + s = s.replaceAll("Table Properties:[^\n]+\n", ""); + // System.out.println("CLEAN: " + s); + return s; + } + + public static void setPathReplacer(Function replacer) { + PATH_REPLACER = replacer; + } + + public static String hash(String s) { + s = PATH_REPLACER.apply(s); + log.debug("PATH REPLACED " + s); + return Hashing.md5().hashString(s, Charset.forName("US-ASCII")).toString(); + } + */ +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppEndEvent.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppEndEvent.java new file mode 100644 index 00000000000000..4e06d2fd923e51 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppEndEvent.java @@ -0,0 +1,46 @@ +package com.linkedin.datahub.lineage.spark.model; + +import java.util.Arrays; +import java.util.List; + +import com.linkedin.common.urn.DataFlowUrn; +import com.linkedin.data.template.StringMap; +import com.linkedin.datahub.lineage.spark.interceptor.LineageUtils; +import com.linkedin.datajob.DataFlowInfo; +import com.linkedin.events.metadata.ChangeType; +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.Getter; +import lombok.ToString; + +@ToString +@Getter +public class AppEndEvent extends LineageEvent { + + private final AppStartEvent start; + + public AppEndEvent(String master, String appName, String appId, long time, AppStartEvent start) { + super(master, appName, appId, time); + this.start = start; + } + + @Override + public List toMcps() { + DataFlowUrn flowUrn = LineageUtils.flowUrn(getMaster(), getAppName()); + + StringMap customProps = start.customProps(); + customProps.put("completedAt", timeStr()); + + DataFlowInfo flowInfo = new DataFlowInfo() + .setName(getAppName()) + .setCustomProperties(customProps); + + MetadataChangeProposal mcpFlowInfo = new MetadataChangeProposal(); + mcpFlowInfo.setAspectName("dataFlowInfo"); + mcpFlowInfo.setAspect(LineageUtils.serializeAspect(flowInfo)); + mcpFlowInfo.setEntityUrn(flowUrn); + mcpFlowInfo.setEntityType("dataFlow"); + mcpFlowInfo.setChangeType(ChangeType.UPSERT); + return Arrays.asList(mcpFlowInfo); + } +} \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppStartEvent.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppStartEvent.java new file mode 100644 index 00000000000000..97c5ecec99f086 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/AppStartEvent.java @@ -0,0 +1,52 @@ +package com.linkedin.datahub.lineage.spark.model; + +import java.util.Arrays; +import java.util.List; + +import com.linkedin.common.urn.DataFlowUrn; +import com.linkedin.data.template.StringMap; +import com.linkedin.datahub.lineage.spark.interceptor.LineageUtils; +import com.linkedin.datajob.DataFlowInfo; +import com.linkedin.events.metadata.ChangeType; +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.Getter; +import lombok.ToString; + +@ToString +@Getter +public class AppStartEvent extends LineageEvent { + + private final String sparkUser; + + public AppStartEvent(String master, String appName, String appId, long time, String sparkUser) { + super(master, appName, appId, time); + this.sparkUser = sparkUser; + } + + @Override + public List toMcps() { + DataFlowUrn flowUrn = LineageUtils.flowUrn(getMaster(), getAppName()); + + DataFlowInfo flowInfo = new DataFlowInfo() + .setName(getAppName()) + .setCustomProperties(customProps()); + + MetadataChangeProposal mcpFlowInfo = new MetadataChangeProposal(); + mcpFlowInfo.setAspectName("dataFlowInfo"); + mcpFlowInfo.setAspect(LineageUtils.serializeAspect(flowInfo)); + mcpFlowInfo.setEntityUrn(flowUrn); + mcpFlowInfo.setEntityType("dataFlow"); + mcpFlowInfo.setChangeType(ChangeType.UPSERT); + return Arrays.asList(mcpFlowInfo); + } + + StringMap customProps() { + StringMap customProps = new StringMap(); + customProps.put("startedAt", timeStr()); + customProps.put("appId", getAppId()); + customProps.put("appName", getAppName()); + customProps.put("sparkUser", sparkUser); + return customProps; + } +} \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/DatasetLineage.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/DatasetLineage.java new file mode 100644 index 00000000000000..82446cee80adb0 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/DatasetLineage.java @@ -0,0 +1,35 @@ +package com.linkedin.datahub.lineage.spark.model; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import com.linkedin.datahub.lineage.spark.model.dataset.SparkDataset; + +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.ToString; + +@RequiredArgsConstructor +@ToString +public class DatasetLineage { + + private final Set sources = new HashSet<>(); + + @Getter + private final String callSiteShort; + + @Getter + private final String plan; + + @Getter + private final SparkDataset sink; + + public void addSource(SparkDataset source) { + sources.add(source); + } + + public Set getSources() { + return Collections.unmodifiableSet(sources); + } +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageConsumer.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageConsumer.java new file mode 100644 index 00000000000000..1c9341116b2b1e --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageConsumer.java @@ -0,0 +1,6 @@ +package com.linkedin.datahub.lineage.spark.model; + +import io.netty.util.internal.shaded.org.jctools.queues.MessagePassingQueue.Consumer; + +public interface LineageConsumer extends Consumer { +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageEvent.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageEvent.java new file mode 100644 index 00000000000000..dcfa514329fa4d --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/LineageEvent.java @@ -0,0 +1,22 @@ +package com.linkedin.datahub.lineage.spark.model; + +import java.util.Date; +import java.util.List; + +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.Data; + +@Data +public abstract class LineageEvent { + private final String master; + private final String appName; + private final String appId; + private final long time; + + public abstract List toMcps(); + + protected String timeStr() { + return new Date(getTime()).toInstant().toString(); + } +} \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecEndEvent.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecEndEvent.java new file mode 100644 index 00000000000000..f5fa389e605d1c --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecEndEvent.java @@ -0,0 +1,47 @@ +package com.linkedin.datahub.lineage.spark.model; + +import java.util.Arrays; +import java.util.List; + +import com.linkedin.common.urn.DataJobUrn; +import com.linkedin.data.template.StringMap; +import com.linkedin.datahub.lineage.spark.interceptor.LineageUtils; +import com.linkedin.datajob.DataJobInfo; +import com.linkedin.events.metadata.ChangeType; +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.Getter; +import lombok.ToString; + +@ToString +@Getter +public class SQLQueryExecEndEvent extends LineageEvent { + + private final long sqlQueryExecId; + private final SQLQueryExecStartEvent start; + + public SQLQueryExecEndEvent(String master, String appName, String appId, long time, long sqlQueryExecId, SQLQueryExecStartEvent start) { + super(master, appName, appId, time); + this.sqlQueryExecId = sqlQueryExecId; + this.start = start; + } + + @Override + public List toMcps() { + DataJobUrn jobUrn = start.jobUrn(); + StringMap customProps = start.customProps(); + customProps.put("completedAt", timeStr()); + + DataJobInfo jobInfo = start.jobInfo() + .setCustomProperties(customProps); + + MetadataChangeProposal mcpJobInfo = new MetadataChangeProposal(); + mcpJobInfo.setAspectName("dataJobInfo"); + mcpJobInfo.setAspect(LineageUtils.serializeAspect(jobInfo)); + mcpJobInfo.setEntityUrn(jobUrn); + mcpJobInfo.setEntityType("dataJob"); + mcpJobInfo.setChangeType(ChangeType.UPSERT); + + return Arrays.asList(mcpJobInfo); + } +} \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecStartEvent.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecStartEvent.java new file mode 100644 index 00000000000000..16909afcb69f7d --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/SQLQueryExecStartEvent.java @@ -0,0 +1,119 @@ +package com.linkedin.datahub.lineage.spark.model; + +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import com.linkedin.common.DatasetUrnArray; +import com.linkedin.common.urn.DataFlowUrn; +import com.linkedin.common.urn.DataJobUrn; +import com.linkedin.data.template.StringMap; +import com.linkedin.datahub.lineage.spark.interceptor.LineageUtils; +import com.linkedin.datahub.lineage.spark.model.dataset.SparkDataset; +import com.linkedin.datajob.DataJobInfo; +import com.linkedin.datajob.DataJobInputOutput; +import com.linkedin.datajob.JobStatus; +import com.linkedin.events.metadata.ChangeType; +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.Getter; +import lombok.ToString; + +@ToString +@Getter +public class SQLQueryExecStartEvent extends LineageEvent { + private final long sqlQueryExecId; + private final DatasetLineage datasetLineage; + + public SQLQueryExecStartEvent(String master, String appName, String appId, long time, long sqlQueryExecId, + DatasetLineage datasetLineage) { + super(master, appName, appId, time); + this.sqlQueryExecId = sqlQueryExecId; + this.datasetLineage = datasetLineage; + } + + @Override + public List toMcps() { + DataJobUrn jobUrn = jobUrn(); + MetadataChangeProposal mcpJobIO = new MetadataChangeProposal(); + mcpJobIO.setAspectName("dataJobInputOutput"); + mcpJobIO.setAspect(LineageUtils.serializeAspect(jobIO())); + mcpJobIO.setEntityUrn(jobUrn); + mcpJobIO.setEntityType("dataJob"); + mcpJobIO.setChangeType(ChangeType.UPSERT); + + DataJobInfo jobInfo = jobInfo(); + jobInfo.setCustomProperties(customProps()); + jobInfo.setStatus(JobStatus.IN_PROGRESS); + + MetadataChangeProposal mcpJobInfo = new MetadataChangeProposal(); + mcpJobInfo.setAspectName("dataJobInfo"); + mcpJobInfo.setAspect(LineageUtils.serializeAspect(jobInfo)); + mcpJobInfo.setEntityUrn(jobUrn); + mcpJobInfo.setEntityType("dataJob"); + mcpJobInfo.setChangeType(ChangeType.UPSERT); + + return Arrays.asList(mcpJobIO, mcpJobInfo); + } + + DataJobInfo jobInfo() { + return new DataJobInfo() + .setName(datasetLineage.getCallSiteShort()) + .setType(DataJobInfo.Type.create("sparkJob")); + } + + DataJobUrn jobUrn() { + /* This is for generating urn from a hash of the plan */ + /* + * Set sourceUrns = datasetLineage.getSources() .parallelStream() .map(x + * -> x.urn().toString()) .collect(Collectors.toSet()); sourceUrns = new + * TreeSet<>(sourceUrns); //sort for consistency + * + * String sinkUrn = datasetLineage.getSink().urn().toString(); String plan = + * LineageUtils.scrubPlan(datasetLineage.getPlan()); String id = + * Joiner.on(",").join(sinkUrn, sourceUrns, plan); + * + * return new DataJobUrn(flowUrn(), "planHash_" + LineageUtils.hash(id)); + */ + return new DataJobUrn(flowUrn(), "QueryExecId_" + sqlQueryExecId); + } + + DataFlowUrn flowUrn() { + return LineageUtils.flowUrn(getMaster(), getAppName()); + } + + StringMap customProps() { + StringMap customProps = new StringMap(); + customProps.put("startedAt", timeStr()); + customProps.put("description", datasetLineage.getCallSiteShort()); + customProps.put("SQLQueryId", Long.toString(sqlQueryExecId)); + customProps.put("appId", getAppId()); + customProps.put("appName", getAppName()); + customProps.put("queryPlan", datasetLineage.getPlan()); + return customProps; + } + + private DataJobInputOutput jobIO() { + DatasetUrnArray out = new DatasetUrnArray(); + out.add(datasetLineage.getSink().urn()); + + DatasetUrnArray in = new DatasetUrnArray(); + + Set sources = new TreeSet<>(new Comparator() { + @Override + public int compare(SparkDataset x, SparkDataset y) { + return x.urn().toString().compareTo(y.urn().toString()); + } + + }); + sources.addAll(datasetLineage.getSources()); // maintain ordering + for (SparkDataset source : sources) { + in.add(source.urn()); + } + + DataJobInputOutput io = new DataJobInputOutput().setInputDatasets(in).setOutputDatasets(out); + return io; + } +} \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/CatalogTableDataset.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/CatalogTableDataset.java new file mode 100644 index 00000000000000..1903d541352055 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/CatalogTableDataset.java @@ -0,0 +1,29 @@ +package com.linkedin.datahub.lineage.spark.model.dataset; + +import org.apache.spark.sql.catalyst.catalog.CatalogTable; + +import com.linkedin.common.FabricType; +import com.linkedin.common.urn.DataPlatformUrn; +import com.linkedin.common.urn.DatasetUrn; + +import lombok.EqualsAndHashCode; +import lombok.ToString; + +@EqualsAndHashCode +@ToString +public class CatalogTableDataset implements SparkDataset { + private final DatasetUrn urn; + + public CatalogTableDataset(CatalogTable table) { + this(table.qualifiedName()); + } + + public CatalogTableDataset(String dsName) { + this.urn = new DatasetUrn(new DataPlatformUrn("hive"), dsName, FabricType.PROD); + } + + @Override + public DatasetUrn urn() { + return this.urn; + } +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/HdfsPathDataset.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/HdfsPathDataset.java new file mode 100644 index 00000000000000..1dec3423ebea75 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/HdfsPathDataset.java @@ -0,0 +1,31 @@ +package com.linkedin.datahub.lineage.spark.model.dataset; + +import org.apache.hadoop.fs.Path; + +import com.linkedin.common.FabricType; +import com.linkedin.common.urn.DataPlatformUrn; +import com.linkedin.common.urn.DatasetUrn; + +import lombok.EqualsAndHashCode; +import lombok.ToString; + +@EqualsAndHashCode +@ToString +public class HdfsPathDataset implements SparkDataset { + private final DatasetUrn urn; + + public HdfsPathDataset(Path path) { + // TODO check static partitions? + this(path.toUri().toString()); + } + + public HdfsPathDataset(String pathUri) { + // TODO check static partitions? + this.urn = new DatasetUrn(new DataPlatformUrn("hdfs"), pathUri, FabricType.PROD); + } + + @Override + public DatasetUrn urn() { + return this.urn; + } +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/JdbcDataset.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/JdbcDataset.java new file mode 100644 index 00000000000000..6d33578743c152 --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/JdbcDataset.java @@ -0,0 +1,39 @@ +package com.linkedin.datahub.lineage.spark.model.dataset; + +import com.linkedin.common.FabricType; +import com.linkedin.common.urn.DataPlatformUrn; +import com.linkedin.common.urn.DatasetUrn; + +import lombok.EqualsAndHashCode; +import lombok.ToString; + +@EqualsAndHashCode +@ToString +public class JdbcDataset implements SparkDataset { + private final DatasetUrn urn; + + public JdbcDataset(String url, String tbl) { + this.urn = new DatasetUrn(new DataPlatformUrn(platformName(url)), dsName(url, tbl), FabricType.PROD); + } + + @Override + public DatasetUrn urn() { + return this.urn; + } + + private static String platformName(String url) { + if (url.contains("postgres")) { + return "postgres"; + } + return "unknownJdbc"; + } + + private static String dsName(String url, String tbl) { + url = url.replaceFirst("jdbc:", ""); + if (url.contains("postgres")) { + url = url.substring(url.lastIndexOf('/') + 1); + } + // TODO different DBs have different formats. TBD mapping to data source names + return url + "." + tbl; + } +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/SparkDataset.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/SparkDataset.java new file mode 100644 index 00000000000000..5da4dc2debfbcd --- /dev/null +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/model/dataset/SparkDataset.java @@ -0,0 +1,7 @@ +package com.linkedin.datahub.lineage.spark.model.dataset; + +import com.linkedin.common.urn.DatasetUrn; + +public interface SparkDataset { + DatasetUrn urn(); +} diff --git a/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java b/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java new file mode 100644 index 00000000000000..ba470fcad191e4 --- /dev/null +++ b/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java @@ -0,0 +1,522 @@ +package com.linkedin.datahub.lineage; + +import static com.github.tomakehurst.wiremock.client.WireMock.ok; +import static com.github.tomakehurst.wiremock.client.WireMock.post; +import static com.github.tomakehurst.wiremock.client.WireMock.postRequestedFor; +import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; +import static com.github.tomakehurst.wiremock.client.WireMock.verify; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.sql.Connection; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestRule; +import org.junit.rules.TestWatcher; +import org.junit.runner.Description; +import org.testcontainers.containers.PostgreSQLContainer; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.tomakehurst.wiremock.WireMockServer; +import com.github.tomakehurst.wiremock.client.MappingBuilder; +import com.github.tomakehurst.wiremock.client.WireMock; +import com.github.tomakehurst.wiremock.core.Admin; +import com.github.tomakehurst.wiremock.core.WireMockConfiguration; +import com.github.tomakehurst.wiremock.extension.Parameters; +import com.github.tomakehurst.wiremock.extension.PostServeAction; +import com.github.tomakehurst.wiremock.matching.MatchResult; +import com.github.tomakehurst.wiremock.matching.StringValuePattern; +import com.github.tomakehurst.wiremock.stubbing.ServeEvent; +import com.linkedin.datahub.lineage.spark.interceptor.LineageUtils; +import com.linkedin.datahub.lineage.spark.model.DatasetLineage; +import com.linkedin.datahub.lineage.spark.model.LineageConsumer; +import com.linkedin.datahub.lineage.spark.model.LineageEvent; +import com.linkedin.datahub.lineage.spark.model.SQLQueryExecStartEvent; +import com.linkedin.datahub.lineage.spark.model.dataset.CatalogTableDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.HdfsPathDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.JdbcDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.SparkDataset; + + + +public class TestSparkJobsLineage { + private static final boolean MOCK_GMS = Boolean.valueOf("true"); // if false, MCPs get written to real GMS server (see GMS_PORT) + private static final boolean VERIFY_EXPECTED = MOCK_GMS && Boolean.valueOf("true"); // if false, "expected" JSONs are overwritten. + + private static final String APP_NAME = "sparkTestApp"; + + private static final String RESOURCE_DIR = "src/test/resources"; + private static final String DATA_DIR = RESOURCE_DIR + "/data"; + private static final String WAREHOUSE_LOC = DATA_DIR + "/hive/warehouse"; + private static final String TEST_DB = "sparktestdb"; + + private static final String MASTER = "local"; + + private static final int N = 3; // num of GMS requests per spark job + + private static final int GMS_PORT = MOCK_GMS ? 8089 : 8080; + + private static final String EXPECTED_JSON_ROOT = "src/test/resources/expected/"; + + private static SparkSession spark; + private static Properties jdbcConnnProperties; + private static DatasetLineageAccumulator acc; + + @SuppressWarnings("rawtypes") + private static final class McpContentPattern extends StringValuePattern { + + public McpContentPattern(String expectedValue) { + super(expectedValue); + } + + // dataflow case, we do not match against expected string + public McpContentPattern() { + super(""); + } + + @Override + public MatchResult match(String actual) { + if (actual.contains("dataJobInputOutput")) { + return expectedValue.contains(relPaths(actual)) ? MatchResult.exactMatch() : MatchResult.noMatch(); + } + try { + HashMap body = new ObjectMapper().readValue(actual, HashMap.class); + HashMap proposal = (HashMap) body.get("proposal"); + String aspectName = (String) proposal.get("aspectName"); + if (aspectName.equals("dataFlowInfo")) { + return checkFlowInfo(proposal); + } + if (actual.contains("dataJobInfo")) { + return checkJobInfo(proposal); + } + return MatchResult.noMatch(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private MatchResult checkJobInfo(HashMap proposal) { + // TODO check custom props etc. + return MatchResult.exactMatch(); + } + + private MatchResult checkFlowInfo(HashMap proposal) { + // TODO check custom props etc. + return MatchResult.exactMatch(); + } + + } + + private static String relPaths(String s) { + return s.replaceAll("file:[0-9|a-z|A-Z|\\-|\\/|_|\\.]*" + RESOURCE_DIR, "file:/" + RESOURCE_DIR); + } + + private static final class RequestFileWriter extends PostServeAction { + + @Override + public String getName() { + return "writeReqJson"; + } + + @SuppressWarnings({ "rawtypes", "unused" }) + public void doAction(ServeEvent serveEvent, Admin admin, Parameters parameters) { + String currentTestRes = parameters.getString("filename"); + if (currentTestRes == null || VERIFY_EXPECTED) { + return; + } + + Path p = Paths.get(EXPECTED_JSON_ROOT, currentTestRes); + System.out.println("Writing json to file " + p); + String json = serveEvent.getRequest().getBodyAsString(); + try { + HashMap body = new ObjectMapper().readValue(json, HashMap.class); + HashMap proposal = (HashMap) body.get("proposal"); + String aspectName = (String) proposal.get("aspectName"); + + // this effectively checks URNs and lineages; other jobInfo/flowInfo are checked + // in McpContentPattern + if (aspectName.equals("dataJobInputOutput")) { + json = relPaths(json); + Files.write(p, Collections.singletonList(json), StandardOpenOption.CREATE, StandardOpenOption.APPEND); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + private static class DatasetLineageAccumulator implements LineageConsumer { + + private final List lineages = new ArrayList<>(); + + public void flushJobs() { + lineages.clear(); + } + + public List getLineages() { + return Collections.unmodifiableList(lineages); + } + + @Override + public void accept(LineageEvent e) { + if (e instanceof SQLQueryExecStartEvent) { + lineages.add(((SQLQueryExecStartEvent) e).getDatasetLineage()); + } + } + } + + @Rule + public TestRule watcher = new TestWatcher() { + protected void starting(Description description) { + if (!MOCK_GMS) { + return; + } + String currentTestRes = description.getMethodName() + ".json"; + + MappingBuilder mapping = baseMapping(); + + try { + if (VERIFY_EXPECTED) { + // setup the request body that we expect + List expected = Files.readAllLines(Paths.get(EXPECTED_JSON_ROOT, currentTestRes)); + mapping.withRequestBody(new McpContentPattern(expected.toString())); + } else { + // overwrite "expected" json file with the MCP request bodies + mapping.withPostServeAction("writeReqJson", Parameters.one("filename", currentTestRes)); + Files.deleteIfExists(Paths.get(EXPECTED_JSON_ROOT, currentTestRes)); + } + } catch (IOException e) { + e.printStackTrace(); + } + wireMock.stubFor(mapping); + } + + @Override + protected void finished(Description description) { + if (!VERIFY_EXPECTED) { + return; + } + assertTrue(WireMock.findUnmatchedRequests().isEmpty()); + wireMock.resetRequests(); + wireMock.resetMappings(); + super.finished(description); + } + }; + + public static WireMockServer wireMock = new WireMockServer(WireMockConfiguration + .options() + .port(8089) + .extensions(new RequestFileWriter())); + + @ClassRule + public static PostgreSQLContainer db = new PostgreSQLContainer<>("postgres:9.6.12") + .withDatabaseName("sparktestdb"); + + private static MappingBuilder baseMapping() { + return post("/aspects?action=ingestProposal") + .willReturn(ok() + .withBody("SUCCESS")); + } + + @BeforeClass + public static void setup() { + + acc = new DatasetLineageAccumulator(); + LineageUtils.registerConsumer("accumulator", acc); + + wireMock.start(); + WireMock.configureFor("localhost", 8089); + MappingBuilder mapping = baseMapping(); + if (VERIFY_EXPECTED) { + mapping.withRequestBody(new McpContentPattern()); + } + wireMock.stubFor(mapping); + + spark = SparkSession + .builder() + .appName(APP_NAME) + .config("spark.master", MASTER) + .config("spark.extraListeners", + "com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter") + .config("spark.datahub.lineage.consumerTypes", "accumulator, mcpEmitter") + .config("spark.datahub.lineage.mcpEmitter.gmsUrl", "http://localhost:" + GMS_PORT) + .config("spark.sql.warehouse.dir", new File(WAREHOUSE_LOC).getAbsolutePath()) + .enableHiveSupport() + .getOrCreate(); + + spark.sql("drop database if exists " + TEST_DB + " cascade"); + spark.sql("create database " + TEST_DB); + jdbcConnnProperties = new Properties(); + jdbcConnnProperties.put("user", db.getUsername()); + jdbcConnnProperties.put("password", db.getPassword()); + + if (VERIFY_EXPECTED) { + verify(1, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + assertTrue(WireMock.findUnmatchedRequests().isEmpty()); + } + wireMock.resetRequests(); + wireMock.resetMappings(); + } + + @AfterClass + public static void tearDown() throws Exception { + wireMock.resetRequests(); + wireMock.resetMappings(); + MappingBuilder mapping = baseMapping(); + if (VERIFY_EXPECTED) { + mapping.withRequestBody(new McpContentPattern()); + } + wireMock.stubFor(mapping); + spark.stop(); + if (VERIFY_EXPECTED) { + verify(1, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + assertTrue(WireMock.findUnmatchedRequests().isEmpty()); + } + wireMock.stop(); + } + + @Before + public void before() { + acc.flushJobs(); + } + + @Test + public void testHdfsInOut() throws Exception { + + Dataset df1 = spark.read().option("header", "true").csv(DATA_DIR + "/in1.csv"); + Dataset df2 = spark.read().option("header", "true").csv(DATA_DIR + "/in2.csv"); + df1.createOrReplaceTempView("v1"); + df2.createOrReplaceTempView("v2"); + + Dataset df = spark + .sql("select v1.c1 as a, v1.c2 as b, v2.c1 as c, v2.c2 as d from v1 join v2 on v1.id = v2.id"); + + // InsertIntoHadoopFsRelationCommand + df.write().mode(SaveMode.Overwrite).csv(DATA_DIR + "/out.csv"); + Thread.sleep(5000); + check(dsl(hdfsDs("out.csv"), hdfsDs("in1.csv"), hdfsDs("in2.csv")), acc.getLineages().get(0)); + if (VERIFY_EXPECTED) { + verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + } + } + + @Test + public void testHdfsInJdbcOut() throws Exception { + Dataset df1 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b"); + + Dataset df2 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "c").withColumnRenamed("c2", "d"); + + Dataset df = df1.join(df2, "id").drop("id"); + + // SaveIntoDataSourceCommand + // HadoopFsRelation input + df.write().mode(SaveMode.Overwrite).jdbc( + db.getJdbcUrl(), + "foo1", jdbcConnnProperties); + Thread.sleep(5000); + check(dsl(pgDs("foo1"), hdfsDs("in1.csv"), hdfsDs("in2.csv")), acc.getLineages().get(0)); + if (VERIFY_EXPECTED) { + verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + } + } + + @Test + public void testHdfsJdbcInJdbcOut() throws Exception { + Connection c = db.createConnection(""); + c.createStatement().execute("create table foo2 (a varchar(5), b int);"); + c.createStatement().execute("insert into foo2 values('a', 4);"); + c.close(); + + Dataset df1 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b2"); + + Dataset df2 = spark.read() + .jdbc(db.getJdbcUrl(), "foo2", jdbcConnnProperties); + + Dataset df = df1.join(df2, "a"); + + // SaveIntoDataSourceCommand + // JDBCRelation input + df.write().mode(SaveMode.Overwrite).jdbc( + db.getJdbcUrl(), + "foo3", jdbcConnnProperties); + Thread.sleep(5000); + check(dsl(pgDs("foo3"), hdfsDs("in1.csv"), pgDs("foo2")), acc.getLineages().get(0)); + if (VERIFY_EXPECTED) { + verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + } + } + + @Test + public void testHdfsInHiveOut() throws Exception { + Dataset df1 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b"); + + Dataset df2 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "c").withColumnRenamed("c2", "d"); + + Dataset df = df1.join(df2, "id").drop("id"); + + df.write().mode(SaveMode.Overwrite).saveAsTable(tbl("foo4")); // CreateDataSourceTableAsSelectCommand + df.write().mode(SaveMode.Append).saveAsTable(tbl("foo4")); // CreateDataSourceTableAsSelectCommand + df.write().insertInto(tbl("foo4")); // InsertIntoHadoopFsRelationCommand + + Thread.sleep(5000); + // TODO same data accessed as Hive Table or Path URI ?? + + DatasetLineage exp = dsl(catTblDs("foo4"), hdfsDs("in1.csv"), hdfsDs("in2.csv")); + check(Collections.nCopies(3, exp), acc.getLineages()); + if (VERIFY_EXPECTED) { + verify(3 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + } + } + + @Test + public void testHiveInHiveOut() throws Exception { + Dataset df1 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b"); + + Dataset df2 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "c").withColumnRenamed("c2", "d"); + + df1.createOrReplaceTempView("v1"); + df2.createOrReplaceTempView("v2"); + + // CreateHiveTableAsSelectCommand + spark.sql("create table " + tbl("foo5") + " as " + + "(select v1.a, v1.b, v2.c, v2.d from v1 join v2 on v1.id = v2.id)"); + + check(dsl(catTblDs("foo5"), hdfsDs("in1.csv"), hdfsDs("in2.csv")), acc.getLineages().get(0)); + + // CreateHiveTableAsSelectCommand + spark.sql("create table " + tbl("hivetab") + " as " + + "(select * from " + tbl("foo5") + ")"); + + check(dsl(catTblDs("hivetab"), catTblDs("foo5")), acc.getLineages().get(1)); + + // InsertIntoHiveTable + spark.sql("insert into " + tbl("hivetab") + " (select * from " + tbl("foo5") + ")"); + check(dsl(catTblDs("hivetab"), catTblDs("foo5")), acc.getLineages().get(2)); + + Dataset df = spark.sql("select * from " + tbl("foo5")); + + // InsertIntoHiveTable + df.write().insertInto(tbl("hivetab")); + Thread.sleep(5000); + check(dsl(catTblDs("hivetab"), catTblDs("foo5")), acc.getLineages().get(3)); + if (VERIFY_EXPECTED) { + verify(4 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + } + } + + @Test + public void testHdfsJdbcInJdbcOutTwoLevel() throws Exception { + Connection c = db.createConnection(""); + c.createStatement().execute("create table foo6 (a varchar(5), b int);"); + c.createStatement().execute("insert into foo6 values('a', 4);"); + c.close(); + + Dataset df1 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b2"); + + Dataset df2 = spark.read() + .jdbc(db.getJdbcUrl(), "foo6", jdbcConnnProperties); + + Dataset df3 = spark.read() + .option("header", "true").csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b3"); + + Dataset df = df1.join(df2, "a").drop("id").join(df3, "a"); + + // SaveIntoDataSourceCommand + // JDBCRelation input + df.write().mode(SaveMode.Overwrite).jdbc( + db.getJdbcUrl(), + "foo7", jdbcConnnProperties); + Thread.sleep(5000); + check(dsl(pgDs("foo7"), hdfsDs("in1.csv"), hdfsDs("in2.csv"), pgDs("foo6")), acc.getLineages().get(0)); + if (VERIFY_EXPECTED) { + verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + } + } + + private static void check(List expected, List actual) { + assertEquals(expected.size(), actual.size()); + for (int i = 0; i < expected.size(); i++) { + check(expected.get(i), actual.get(i)); + } + } + + private static void check(DatasetLineage expected, DatasetLineage actual) { + assertEquals(expected.getSink().toString(), actual.getSink().toString()); + assertEquals(dsToStrings(expected.getSources()), dsToStrings(actual.getSources())); + assertTrue(actual.getCallSiteShort().contains("TestSparkJobsLineage")); + } + + private static Set dsToStrings(Set datasets) { + return datasets.stream().map(x -> x.toString()).collect(Collectors.toSet()); + } + + private static DatasetLineage dsl(SparkDataset sink, SparkDataset... source) { + return dsl(null, sink, source); + } + + private static DatasetLineage dsl(String callSite, SparkDataset sink, SparkDataset... source) { + DatasetLineage lineage = new DatasetLineage(callSite, "unknownPlan", sink); + Arrays.asList(source).forEach(x -> lineage.addSource(x)); + return lineage; + } + + private static HdfsPathDataset hdfsDs(String fileName) { + return new HdfsPathDataset("file:" + abs(DATA_DIR + "/" + fileName)); + } + + private static JdbcDataset pgDs(String tbl) { + return new JdbcDataset(db.getJdbcUrl(), tbl); + } + + private static CatalogTableDataset catTblDs(String tbl) { + return new CatalogTableDataset(tbl(tbl)); + } + + private static String tbl(String tbl) { + return TEST_DB + "." + tbl; + } + + private static String abs(String relPath) { + return new File(relPath).getAbsolutePath(); + } +} \ No newline at end of file diff --git a/spark-lineage/src/test/resources/data/in1.csv/part1.csv b/spark-lineage/src/test/resources/data/in1.csv/part1.csv new file mode 100644 index 00000000000000..b65449abf61931 --- /dev/null +++ b/spark-lineage/src/test/resources/data/in1.csv/part1.csv @@ -0,0 +1,3 @@ +id,c1,c2 +1,a,4 +2,a,5 diff --git a/spark-lineage/src/test/resources/data/in2.csv/part1.csv b/spark-lineage/src/test/resources/data/in2.csv/part1.csv new file mode 100644 index 00000000000000..cd1853d694a0d2 --- /dev/null +++ b/spark-lineage/src/test/resources/data/in2.csv/part1.csv @@ -0,0 +1,4 @@ +id,c1,c2 +1,a,4 +2,b,5 +3,b,6 diff --git a/spark-lineage/src/test/resources/expected/testHdfsInHiveOut.json b/spark-lineage/src/test/resources/expected/testHdfsInHiveOut.json new file mode 100644 index 00000000000000..b327969620aeab --- /dev/null +++ b/spark-lineage/src/test/resources/expected/testHdfsInHiveOut.json @@ -0,0 +1,3 @@ +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_9)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in2.csv,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.foo4,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_10)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in2.csv,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.foo4,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_11)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in2.csv,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.foo4,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} diff --git a/spark-lineage/src/test/resources/expected/testHdfsInJdbcOut.json b/spark-lineage/src/test/resources/expected/testHdfsInJdbcOut.json new file mode 100644 index 00000000000000..e89db8b2b82395 --- /dev/null +++ b/spark-lineage/src/test/resources/expected/testHdfsInJdbcOut.json @@ -0,0 +1 @@ +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_24)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in2.csv,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:postgres,sparktestdb.foo1,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} diff --git a/spark-lineage/src/test/resources/expected/testHdfsInOut.json b/spark-lineage/src/test/resources/expected/testHdfsInOut.json new file mode 100644 index 00000000000000..3da8ec90028789 --- /dev/null +++ b/spark-lineage/src/test/resources/expected/testHdfsInOut.json @@ -0,0 +1 @@ +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_6)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in2.csv,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/out.csv,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} diff --git a/spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOut.json b/spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOut.json new file mode 100644 index 00000000000000..06ea7714670700 --- /dev/null +++ b/spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOut.json @@ -0,0 +1 @@ +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_13)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:postgres,sparktestdb.foo2,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:postgres,sparktestdb.foo3,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} diff --git a/spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOutTwoLevel.json b/spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOutTwoLevel.json new file mode 100644 index 00000000000000..763b3319a0511d --- /dev/null +++ b/spark-lineage/src/test/resources/expected/testHdfsJdbcInJdbcOutTwoLevel.json @@ -0,0 +1 @@ +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_27)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in2.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:postgres,sparktestdb.foo6,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:postgres,sparktestdb.foo7,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} diff --git a/spark-lineage/src/test/resources/expected/testHiveInHiveOut.json b/spark-lineage/src/test/resources/expected/testHiveInHiveOut.json new file mode 100644 index 00000000000000..9cd7268a203813 --- /dev/null +++ b/spark-lineage/src/test/resources/expected/testHiveInHiveOut.json @@ -0,0 +1,4 @@ +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_18)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in1.csv,PROD)\",\"urn:li:dataset:(urn:li:dataPlatform:hdfs,file:/src/test/resources/data/in2.csv,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.foo5,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_19)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.foo5,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.hivetab,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_20)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.foo5,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.hivetab,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} +{ "proposal" :{"aspectName":"dataJobInputOutput","entityUrn":"urn:li:dataJob:(urn:li:dataFlow:(spark,sparkTestApp,local),QueryExecId_21)","entityType":"dataJob","aspect":{"value":"{\"inputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.foo5,PROD)\"],\"outputDatasets\":[\"urn:li:dataset:(urn:li:dataPlatform:hive,sparktestdb.hivetab,PROD)\"]}","contentType":"application/json"},"changeType":"UPSERT"}} diff --git a/spark-lineage/src/test/resources/org/apache/spark/log4j-defaults.properties b/spark-lineage/src/test/resources/org/apache/spark/log4j-defaults.properties new file mode 100644 index 00000000000000..29007560d25313 --- /dev/null +++ b/spark-lineage/src/test/resources/org/apache/spark/log4j-defaults.properties @@ -0,0 +1,8 @@ +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.out +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +log4j.logger.com.linkedin.datahub.lineage=DEBUG \ No newline at end of file From 16f7ef7d9350194cfde41e9b6fc1960b5fc281c6 Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Wed, 8 Dec 2021 20:22:10 +0530 Subject: [PATCH 02/24] fix(spark-lineage): Adding missing file in commit --- settings.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/settings.gradle b/settings.gradle index 36d3b3d8c499b4..f654dd1275ebbe 100644 --- a/settings.gradle +++ b/settings.gradle @@ -39,3 +39,4 @@ include 'metadata-perf' include 'docs-website' include 'metadata-models-custom' include 'entity-registry:custom-test-model' +include 'spark-lineage' From f8054cb9db6a01fc352bb93eb424045b85989a92 Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Wed, 8 Dec 2021 20:29:44 +0530 Subject: [PATCH 03/24] docs(spark lineage): modify to fit under "Lineage" category --- spark-lineage/README.md | 34 ++++++++++++++-------------------- 1 file changed, 14 insertions(+), 20 deletions(-) diff --git a/spark-lineage/README.md b/spark-lineage/README.md index 0a640068d21540..89ef07d45df564 100644 --- a/spark-lineage/README.md +++ b/spark-lineage/README.md @@ -1,28 +1,22 @@ -# Metadata Ingestion: Emitters - -Emitters are datahub artifacts that you can plug into your source data platforms. Depending on the capabilities provided by the data platform, an emitter can extract metadata such as datasets, jobs, lineage etc. whenever relevant events occur. The emitter then pushes this metadata to Datahub's GMS using Kafka and/or REST. - -All emitters thus typically expect a minimal configuration to specify the connection details to GMS. - -## Spark lineage emitter +# Spark lineage emitter The Spark lineage emitter is a java library that provides a Spark listener implementation "DatahubLineageEmitter". The DatahubLineageEmitter listens to events such application start/end, and SQLExecution start/end to create pipelines (i.e. DataJob) and tasks (i.e. DataFlow) in Datahub along with lineage. -### Configuring Spark emitter +## Configuring Spark emitter Listener configuration can be done using a config file or while creating a spark Session. -#### Config file for spark-submit +### Config file for spark-submit When running jobs using spark-submit, the listener is to be configured in the config file. ``` spark.master spark://spark-master:7077 -#Configuring datahub spark listner jar +#Configuring datahub spark listener jar spark.jars.packages io.acryl:spark-lineage:0.0.1 spark.extraListeners com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter spark.datahub.lineage.mcpEmitter.gmsUrl http://localhost:8080 ``` -#### Configuring with SparkSession Builder for notebooks +### Configuring with SparkSession Builder for notebooks When running interactive jobs from a notebook, the listener can be configured while building the Spark Session. ```python @@ -36,11 +30,11 @@ spark = SparkSession.builder \ .getOrCreate() ``` -### Model mapping +## Model mapping A pipeline is created per Spark . A task is created per unique Spark query execution within an app. -#### Custom properties & relating to Spark UI +### Custom properties & relating to Spark UI The following custom properties in pipelines and tasks relate to the Spark UI: - appName and appId in a pipeline can be used to determine the Spark application - description and SQLQueryId in a task can be used to determine the Query Execution within the application on the SQL tab of Spark UI @@ -48,23 +42,23 @@ The following custom properties in pipelines and tasks relate to the Spark UI: Other custom properties of pipelines and tasks capture the start and end times of execution etc. The query plan is captured in the *queryPlan* property of a task. -### Release notes for v0.0.1 +## Release notes for v0.0.1 In this version, basic dataset-level lineage is captured using the model mapping as mentioned earlier. -#### Spark versions supported +### Spark versions supported The primary version tested is Spark/Scala version 2.4.8/2_11. We anticipate this to work well with other Spark 2.4.x versions and Scala 2_11. Support for other Spark versions is planned in the very near future. -#### Environments tested with +### Environments tested with This initial release has been tested with the following environments: - spark-submit of Python/Java applications to local and remote servers - notebooks Note that testing for other environments such as Databricks and standalone applications is planned in near future. -#### Spark commands supported +### Spark commands supported Below is a list of Spark commands that are parsed currently: - InsertIntoHadoopFsRelationCommand - SaveIntoDataSourceCommand (jdbc) @@ -73,12 +67,12 @@ Below is a list of Spark commands that are parsed currently: Effectively, these support data sources/sinks corresponding to Hive, HDFS and JDBC. -#### Spark commands not yet supported +### Spark commands not yet supported - View related commands - Cache commands and implications on lineage - RDD jobs -#### Important notes on usage +### Important notes on usage - It is advisable to ensure appName is used appropriately to ensure you can trace lineage from a pipeline back to your source code. @@ -88,7 +82,7 @@ Effectively, these support data sources/sinks corresponding to Hive, HDFS and JD - For HDFS sources, the folder (name) is regarded as the dataset (name) to align with typical storage of parquet/csv formats. -### Known limitations +## Known limitations - Only postgres supported for JDBC sources in this initial release. Support for other driver URL formats will be added in future. - Behavior with cached datasets is not fully specified/defined in context of lineage. - There is a possibility that very short-lived jobs that run within a few milliseconds may not be captured by the listener. This should not cause an issue for realistic Spark applications. From 11fdcdde73f1db7b0be7777d6bcb863744611fe8 Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Wed, 8 Dec 2021 20:33:03 +0530 Subject: [PATCH 04/24] docs(spark lineage): add docs to "Lineage" category --- docs-website/sidebars.js | 1 + 1 file changed, 1 insertion(+) diff --git a/docs-website/sidebars.js b/docs-website/sidebars.js index e31505f76edc40..90b90208236cd2 100644 --- a/docs-website/sidebars.js +++ b/docs-website/sidebars.js @@ -98,6 +98,7 @@ module.exports = { "docs/lineage/airflow", "docker/airflow/local_airflow", "docs/lineage/sample_code", + "spark-lineage/README" ], }, ], From 7c36b33fcefcef7c8a03d9f66b8e6ee4b0197c91 Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Mon, 13 Dec 2021 22:11:34 +0530 Subject: [PATCH 05/24] fix(spark-lineage): handle absence of execution context gracefully --- .../lineage/spark/interceptor/DatahubLineageEmitter.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java index d39d507e5519bb..8c70f4f8acff65 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java @@ -263,6 +263,10 @@ public Void apply(SparkContext sc) { // TODO sqlEvt.details() unused private void processExecution(SparkListenerSQLExecutionStart sqlStart) { QueryExecution queryExec = SQLExecution.getQueryExecution(sqlStart.executionId()); + if (queryExec == null) { + log.error("Skipping processing for sql exec Id" + sqlStart.executionId() + " as Query execution context could not be read from current spark state"); + return; + } LogicalPlan plan = queryExec.optimizedPlan(); SparkSession sess = queryExec.sparkSession(); SparkContext ctx = sess.sparkContext(); From 38c36e23521bcc045707caf7a1bbc668e1f03613 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Mon, 13 Dec 2021 11:32:01 -0800 Subject: [PATCH 06/24] fixing lint --- docs-website/sidebars.js | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs-website/sidebars.js b/docs-website/sidebars.js index 90b90208236cd2..08acfdc1de73e8 100644 --- a/docs-website/sidebars.js +++ b/docs-website/sidebars.js @@ -98,7 +98,7 @@ module.exports = { "docs/lineage/airflow", "docker/airflow/local_airflow", "docs/lineage/sample_code", - "spark-lineage/README" + "spark-lineage/README", ], }, ], From b8b12d916268d74c424340cd4e74c1878e0bf181 Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Thu, 16 Dec 2021 22:58:22 +0530 Subject: [PATCH 07/24] docs(spark-lineage): Update artifact name and version --- spark-lineage/README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/spark-lineage/README.md b/spark-lineage/README.md index 89ef07d45df564..7dbe1dae4841db 100644 --- a/spark-lineage/README.md +++ b/spark-lineage/README.md @@ -11,7 +11,7 @@ When running jobs using spark-submit, the listener is to be configured in the co spark.master spark://spark-master:7077 #Configuring datahub spark listener jar -spark.jars.packages io.acryl:spark-lineage:0.0.1 +spark.jars.packages io.acryl:datahub-spark-lineage:0.0.2 spark.extraListeners com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter spark.datahub.lineage.mcpEmitter.gmsUrl http://localhost:8080 ``` @@ -23,7 +23,7 @@ When running interactive jobs from a notebook, the listener can be configured wh spark = SparkSession.builder \ .master("spark://spark-master:7077") \ .appName("test-application") \ - .config("spark.jars.packages","io.acryl:spark-lineage:0.0.1") \ + .config("spark.jars.packages","io.acryl:datahub-spark-lineage:0.0.2") \ .config("spark.extraListeners","com.linkedin.datahub.lineage.interceptor.spark.DatahubLineageEmitter") \ .config("spark.datahub.lineage.mcpEmitter.gmsUrl", "http://localhost:8080") \ .enableHiveSupport() \ @@ -42,7 +42,7 @@ The following custom properties in pipelines and tasks relate to the Spark UI: Other custom properties of pipelines and tasks capture the start and end times of execution etc. The query plan is captured in the *queryPlan* property of a task. -## Release notes for v0.0.1 +## Release notes for v0.0.2 In this version, basic dataset-level lineage is captured using the model mapping as mentioned earlier. ### Spark versions supported From 890f761275ad83c3a4b61673b0f3252a6d9a81d6 Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Tue, 21 Dec 2021 14:46:52 +0530 Subject: [PATCH 08/24] feat(java datahub-client): add Java REST emitter --- .../java/datahub-client/build.gradle | 24 ++++ .../main/java/datahub/client/RESTEmitter.java | 112 ++++++++++++++++++ settings.gradle | 1 + 3 files changed, 137 insertions(+) create mode 100644 metadata-integration/java/datahub-client/build.gradle create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java diff --git a/metadata-integration/java/datahub-client/build.gradle b/metadata-integration/java/datahub-client/build.gradle new file mode 100644 index 00000000000000..502092853d61f9 --- /dev/null +++ b/metadata-integration/java/datahub-client/build.gradle @@ -0,0 +1,24 @@ + +apply plugin: 'java' +apply plugin: 'com.github.johnrengelman.shadow' + +dependencies { + + compile project(':metadata-models') + compile externalDependency.jacksonDataBind + compileOnly externalDependency.lombok + annotationProcessor externalDependency.lombok +} + +test { + useJUnit() +} + +shadowJar { + zip64=true + classifier='' +} + +assemble { + dependsOn shadowJar +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java new file mode 100644 index 00000000000000..5b900ec9212986 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java @@ -0,0 +1,112 @@ +package datahub.client; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.net.HttpURLConnection; +import java.net.URL; +import java.util.HashMap; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.data.template.JacksonDataTemplateCodec; +import com.linkedin.mxe.MetadataChangeProposal; + +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +@RequiredArgsConstructor +public class RESTEmitter { + + private static final JacksonDataTemplateCodec DATA_TEMPLATE_CODEC = new JacksonDataTemplateCodec(); + + @Getter + private final String gmsUrl; + + @Getter + private final int connectTimeoutSec; + + @Getter + private final int readTimeoutSec; + + @Getter + private final String token; + + public void emit(MetadataChangeProposal mcp) throws IOException { + String payloadJson = DATA_TEMPLATE_CODEC.mapToString(mcp.data()); + ObjectMapper om = new ObjectMapper(); + TypeReference> typeRef = new TypeReference>() { + }; + HashMap o = om.readValue(payloadJson, typeRef); + while (o.values().remove(null)) { + } + payloadJson = om.writeValueAsString(o); + payloadJson = "{" + " \"proposal\" :" + payloadJson + "}"; + log.debug("Emitting payload: " + payloadJson + "\n to URL " + this.gmsUrl + "/aspects?action=ingestProposal"); + this.makeRequest(this.gmsUrl + "/aspects?action=ingestProposal", "POST", payloadJson); + } + + public boolean makeRequest(String urlStr, String method, String payloadJson) throws IOException { + URL url = new URL(urlStr); + HttpURLConnection con = (HttpURLConnection) url.openConnection(); + con.setConnectTimeout(this.connectTimeoutSec * 1000); + con.setReadTimeout(this.connectTimeoutSec * 1000); + con.setRequestMethod(method); + con.setRequestProperty("Content-Type", "application/json"); + con.setRequestProperty("X-RestLi-Protocol-Version", "2.0.0"); + if (this.token != null) { + con.setRequestProperty("Authorization", "Bearer " + token); + } +// con.setRequestProperty("Accept", "application/json"); + con.setDoOutput(true); + if (payloadJson != null) { + try (OutputStream os = con.getOutputStream()) { + byte[] input = payloadJson.getBytes("utf-8"); + os.write(input, 0, input.length); + } + } + try (BufferedReader br = new BufferedReader(new InputStreamReader(con.getInputStream(), "utf-8"))) { + StringBuilder response = new StringBuilder(); + String responseLine = null; + while ((responseLine = br.readLine()) != null) { + response.append(responseLine.trim()); + } + log.debug("URL: " + urlStr + " Response: " + response.toString()); + } + return true; + + } + + public boolean testConnection() { + try { + this.makeRequest(this.gmsUrl + "/config", "GET", null); + return true; + + } catch (IOException e) { + e.printStackTrace(); + return false; + } + } + + public static RESTEmitter create(String gmsUrl) { + // setting default connect time out to 30 seconds. should be plenty to connect + // setting default readTimeoutSec to 30 sec + return new RESTEmitter(gmsUrl, 30, 30, null); + } + + public static RESTEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec) { + return new RESTEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, null); + } + + public static RESTEmitter create(String gmsUrl, String token) { + return new RESTEmitter(gmsUrl, 30, 30, token); + } + + public static RESTEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec, String token) { + return new RESTEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, token); + } + +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index f654dd1275ebbe..67b14f5dfed394 100644 --- a/settings.gradle +++ b/settings.gradle @@ -40,3 +40,4 @@ include 'docs-website' include 'metadata-models-custom' include 'entity-registry:custom-test-model' include 'spark-lineage' +include 'metadata-integration:java:datahub-client' From a508aa175a4347731639555d22af833ae1e7917d Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Tue, 21 Dec 2021 15:08:41 +0530 Subject: [PATCH 09/24] refactor(spark-lineage): use of datahub client for REST based gms communcation --- spark-lineage/build.gradle | 7 +- .../lineage/consumer/impl/MCPEmitter.java | 1 + .../lineage/consumer/impl/RESTEmitter.java | 85 ------------------- .../interceptor/DatahubLineageEmitter.java | 2 +- 4 files changed, 7 insertions(+), 88 deletions(-) delete mode 100644 spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java diff --git a/spark-lineage/build.gradle b/spark-lineage/build.gradle index 660969de23ab10..a641d190bb4f56 100644 --- a/spark-lineage/build.gradle +++ b/spark-lineage/build.gradle @@ -21,9 +21,10 @@ dependencies { compileOnly externalDependency.lombok annotationProcessor externalDependency.lombok - implementation(project(':metadata-models')) { + implementation(project(':metadata-integration:java:datahub-client')) { exclude group: "org.antlr" exclude group: "com.google.guava" // causes issues with Guava Stopwatch constructor + exclude group: "com.fasterxml.jackson.core" } implementation(externalDependency.sparkSql){ @@ -33,7 +34,9 @@ dependencies { exclude group: "org.apache.hadoop" } - testImplementation(externalDependency.postgresql) + testImplementation(externalDependency.postgresql){ + exclude group: "com.fasterxml.jackson.core" + } testImplementation externalDependency.mockito diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java index 017d75c6078375..80bfef236a5dac 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java @@ -13,6 +13,7 @@ import com.linkedin.datahub.lineage.spark.model.LineageEvent; import com.linkedin.mxe.MetadataChangeProposal; +import datahub.client.RESTEmitter; import lombok.extern.slf4j.Slf4j; @Slf4j diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java deleted file mode 100644 index 6fd3ffee83e3b8..00000000000000 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/RESTEmitter.java +++ /dev/null @@ -1,85 +0,0 @@ -package com.linkedin.datahub.lineage.consumer.impl; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.net.HttpURLConnection; -import java.net.URL; -import java.util.HashMap; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.linkedin.data.template.JacksonDataTemplateCodec; -import com.linkedin.mxe.MetadataChangeProposal; - -import lombok.Getter; -import lombok.RequiredArgsConstructor; -import lombok.extern.slf4j.Slf4j; - -@Slf4j -@RequiredArgsConstructor -public class RESTEmitter { - - private static final JacksonDataTemplateCodec DATA_TEMPLATE_CODEC = new JacksonDataTemplateCodec(); - - @Getter - private final String gmsUrl; - - public void emit(MetadataChangeProposal mcp) throws IOException { - String payloadJson = DATA_TEMPLATE_CODEC.mapToString(mcp.data()); - ObjectMapper om = new ObjectMapper(); - TypeReference> typeRef = new TypeReference>() { - }; - HashMap o = om.readValue(payloadJson, typeRef); - while (o.values().remove(null)) { - - } - - payloadJson = om.writeValueAsString(o); - payloadJson = "{" + " \"proposal\" :" + payloadJson + "}"; - log.debug("Emitting payload: " + payloadJson + "\n to URL " + this.gmsUrl + "/aspects?action=ingestProposal"); - RESTEmitter.makeRequest(this.gmsUrl + "/aspects?action=ingestProposal", "POST", payloadJson); - } - - public static boolean makeRequest(String urlStr, String method, String payloadJson) throws IOException { - URL url = new URL(urlStr); - HttpURLConnection con = (HttpURLConnection) url.openConnection(); - con.setRequestMethod(method); - con.setRequestProperty("Content-Type", "application/json"); - con.setRequestProperty("X-RestLi-Protocol-Version", "2.0.0"); -// con.setRequestProperty("Accept", "application/json"); - con.setDoOutput(true); - if (payloadJson != null) { - try (OutputStream os = con.getOutputStream()) { - byte[] input = payloadJson.getBytes("utf-8"); - os.write(input, 0, input.length); - } - } - try (BufferedReader br = new BufferedReader(new InputStreamReader(con.getInputStream(), "utf-8"))) { - StringBuilder response = new StringBuilder(); - String responseLine = null; - while ((responseLine = br.readLine()) != null) { - response.append(responseLine.trim()); - } - log.debug("URL: " + urlStr + " Response: " + response.toString()); - } - return true; - - } - - public boolean testConnection() { - try { - RESTEmitter.makeRequest(this.gmsUrl + "/config", "GET", null); - return true; - - } catch (IOException e) { - e.printStackTrace(); - return false; - } - } - - public static RESTEmitter create(String gmsUrl) { - return new RESTEmitter(gmsUrl); - } -} \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java index 8c70f4f8acff65..72d231f991aa75 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/DatahubLineageEmitter.java @@ -46,7 +46,7 @@ @Slf4j public class DatahubLineageEmitter extends SparkListener { - private static final int THREAD_CNT = 10; + private static final int THREAD_CNT = 16; public static final String CONSUMER_TYPE_KEY = "spark.datahub.lineage.consumerTypes"; private final Map appDetails = new ConcurrentHashMap<>(); From bd67427edb5956e0a3c75f4493789e09f174afbf Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Tue, 21 Dec 2021 20:03:59 +0530 Subject: [PATCH 10/24] fix(spark-lineage): modify gms server url config name --- spark-lineage/README.md | 10 +++++----- .../datahub/lineage/consumer/impl/MCPEmitter.java | 2 +- .../linkedin/datahub/lineage/TestSparkJobsLineage.java | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/spark-lineage/README.md b/spark-lineage/README.md index 7dbe1dae4841db..e679bf1e6cebe4 100644 --- a/spark-lineage/README.md +++ b/spark-lineage/README.md @@ -11,9 +11,9 @@ When running jobs using spark-submit, the listener is to be configured in the co spark.master spark://spark-master:7077 #Configuring datahub spark listener jar -spark.jars.packages io.acryl:datahub-spark-lineage:0.0.2 +spark.jars.packages io.acryl:datahub-spark-lineage:0.0.3 spark.extraListeners com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter -spark.datahub.lineage.mcpEmitter.gmsUrl http://localhost:8080 +spark.datahub.rest.server http://localhost:8080 ``` ### Configuring with SparkSession Builder for notebooks @@ -23,9 +23,9 @@ When running interactive jobs from a notebook, the listener can be configured wh spark = SparkSession.builder \ .master("spark://spark-master:7077") \ .appName("test-application") \ - .config("spark.jars.packages","io.acryl:datahub-spark-lineage:0.0.2") \ + .config("spark.jars.packages","io.acryl:datahub-spark-lineage:0.0.3") \ .config("spark.extraListeners","com.linkedin.datahub.lineage.interceptor.spark.DatahubLineageEmitter") \ - .config("spark.datahub.lineage.mcpEmitter.gmsUrl", "http://localhost:8080") \ + .config("spark.datahub.rest.server", "http://localhost:8080") \ .enableHiveSupport() \ .getOrCreate() ``` @@ -42,7 +42,7 @@ The following custom properties in pipelines and tasks relate to the Spark UI: Other custom properties of pipelines and tasks capture the start and end times of execution etc. The query plan is captured in the *queryPlan* property of a task. -## Release notes for v0.0.2 +## Release notes for v0.0.3 In this version, basic dataset-level lineage is captured using the model mapping as mentioned earlier. ### Spark versions supported diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java index 80bfef236a5dac..60776ba7aea08c 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java @@ -19,7 +19,7 @@ @Slf4j public class MCPEmitter implements LineageConsumer { - private static final String GMS_URL_KEY = "spark.datahub.lineage.mcpEmitter.gmsUrl"; + private static final String GMS_URL_KEY = "spark.datahub.rest.server"; private static final String SENTINEL = "moot"; private ConcurrentHashMap singleton = new ConcurrentHashMap<>(); diff --git a/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java b/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java index ba470fcad191e4..fa908b587731a0 100644 --- a/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java +++ b/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java @@ -262,7 +262,7 @@ public static void setup() { .config("spark.extraListeners", "com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter") .config("spark.datahub.lineage.consumerTypes", "accumulator, mcpEmitter") - .config("spark.datahub.lineage.mcpEmitter.gmsUrl", "http://localhost:" + GMS_PORT) + .config("spark.datahub.rest.server", "http://localhost:" + GMS_PORT) .config("spark.sql.warehouse.dir", new File(WAREHOUSE_LOC).getAbsolutePath()) .enableHiveSupport() .getOrCreate(); From 1f30a587b74ec98b437b410ffb8048d69426444b Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Tue, 21 Dec 2021 20:10:05 +0530 Subject: [PATCH 11/24] fix(datahub-client) : fix readtimeout setter --- .../main/java/datahub/client/RESTEmitter.java | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java index 5b900ec9212986..631d42f6e22561 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java @@ -4,6 +4,8 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.OutputStream; +import java.io.PrintWriter; +import java.io.StringWriter; import java.net.HttpURLConnection; import java.net.URL; import java.util.HashMap; @@ -22,7 +24,10 @@ public class RESTEmitter { private static final JacksonDataTemplateCodec DATA_TEMPLATE_CODEC = new JacksonDataTemplateCodec(); - + private static final int DEFAULT_CONNECT_TIMEOUT_SEC = 30; + private static final int DEFAULT_READ_TIMEOUT_SEC = 30; + + @Getter private final String gmsUrl; @@ -53,14 +58,13 @@ public boolean makeRequest(String urlStr, String method, String payloadJson) thr URL url = new URL(urlStr); HttpURLConnection con = (HttpURLConnection) url.openConnection(); con.setConnectTimeout(this.connectTimeoutSec * 1000); - con.setReadTimeout(this.connectTimeoutSec * 1000); + con.setReadTimeout(this.readTimeoutSec * 1000); con.setRequestMethod(method); con.setRequestProperty("Content-Type", "application/json"); con.setRequestProperty("X-RestLi-Protocol-Version", "2.0.0"); if (this.token != null) { con.setRequestProperty("Authorization", "Bearer " + token); } -// con.setRequestProperty("Accept", "application/json"); con.setDoOutput(true); if (payloadJson != null) { try (OutputStream os = con.getOutputStream()) { @@ -80,21 +84,13 @@ public boolean makeRequest(String urlStr, String method, String payloadJson) thr } - public boolean testConnection() { - try { + public boolean testConnection() throws IOException { this.makeRequest(this.gmsUrl + "/config", "GET", null); return true; - - } catch (IOException e) { - e.printStackTrace(); - return false; - } } public static RESTEmitter create(String gmsUrl) { - // setting default connect time out to 30 seconds. should be plenty to connect - // setting default readTimeoutSec to 30 sec - return new RESTEmitter(gmsUrl, 30, 30, null); + return new RESTEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, null); } public static RESTEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec) { @@ -102,7 +98,7 @@ public static RESTEmitter create(String gmsUrl, int connectTimeoutSec, int readT } public static RESTEmitter create(String gmsUrl, String token) { - return new RESTEmitter(gmsUrl, 30, 30, token); + return new RESTEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, token); } public static RESTEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec, String token) { From 0a4b981170411eaf47670d3e901a0a49521e594a Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Tue, 21 Dec 2021 21:16:26 +0530 Subject: [PATCH 12/24] fix(datahub-client): remove unused imports causing check fail --- .../src/main/java/datahub/client/RESTEmitter.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java index 631d42f6e22561..d76179220e0f80 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java @@ -4,8 +4,6 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.OutputStream; -import java.io.PrintWriter; -import java.io.StringWriter; import java.net.HttpURLConnection; import java.net.URL; import java.util.HashMap; From 61c0c62d40d34261cfeec9c8a2fc8bb01feb4520 Mon Sep 17 00:00:00 2001 From: MugdhaHardikar-GSLab Date: Wed, 22 Dec 2021 14:44:23 +0530 Subject: [PATCH 13/24] refactor(datahub-client): rename RESTEmitter RestEmitter --- .../{RESTEmitter.java => RestEmitter.java} | 18 +++++++++--------- .../impl/{MCPEmitter.java => McpEmitter.java} | 12 ++++++------ .../spark/interceptor/LineageUtils.java | 4 ++-- 3 files changed, 17 insertions(+), 17 deletions(-) rename metadata-integration/java/datahub-client/src/main/java/datahub/client/{RESTEmitter.java => RestEmitter.java} (85%) rename spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/{MCPEmitter.java => McpEmitter.java} (86%) diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java similarity index 85% rename from metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java rename to metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java index d76179220e0f80..0387752c6303d5 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RESTEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java @@ -19,7 +19,7 @@ @Slf4j @RequiredArgsConstructor -public class RESTEmitter { +public class RestEmitter { private static final JacksonDataTemplateCodec DATA_TEMPLATE_CODEC = new JacksonDataTemplateCodec(); private static final int DEFAULT_CONNECT_TIMEOUT_SEC = 30; @@ -87,20 +87,20 @@ public boolean testConnection() throws IOException { return true; } - public static RESTEmitter create(String gmsUrl) { - return new RESTEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, null); + public static RestEmitter create(String gmsUrl) { + return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, null); } - public static RESTEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec) { - return new RESTEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, null); + public static RestEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec) { + return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, null); } - public static RESTEmitter create(String gmsUrl, String token) { - return new RESTEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, token); + public static RestEmitter create(String gmsUrl, String token) { + return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, token); } - public static RESTEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec, String token) { - return new RESTEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, token); + public static RestEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec, String token) { + return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, token); } } \ No newline at end of file diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java similarity index 86% rename from spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java rename to spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java index 60776ba7aea08c..ed057183562513 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/MCPEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java @@ -13,19 +13,19 @@ import com.linkedin.datahub.lineage.spark.model.LineageEvent; import com.linkedin.mxe.MetadataChangeProposal; -import datahub.client.RESTEmitter; +import datahub.client.RestEmitter; import lombok.extern.slf4j.Slf4j; @Slf4j -public class MCPEmitter implements LineageConsumer { +public class McpEmitter implements LineageConsumer { private static final String GMS_URL_KEY = "spark.datahub.rest.server"; private static final String SENTINEL = "moot"; - private ConcurrentHashMap singleton = new ConcurrentHashMap<>(); + private ConcurrentHashMap singleton = new ConcurrentHashMap<>(); private void emit(List mcps) { - RESTEmitter emitter = emitter(); + RestEmitter emitter = emitter(); if (emitter != null) { mcps.forEach(mcp -> { log.debug("Emitting \n" + mcp); @@ -45,13 +45,13 @@ private void emit(List mcps) { // TODO ideally the impl here should not be tied to Spark; the LineageConsumer // API needs tweaking to include configs - private RESTEmitter emitter() { + private RestEmitter emitter() { singleton.computeIfAbsent(SENTINEL, x -> { SparkConf conf = SparkEnv.get().conf(); if (conf.contains(GMS_URL_KEY)) { String gmsUrl = conf.get(GMS_URL_KEY); log.debug("REST emitter configured with GMS url " + gmsUrl); - return RESTEmitter.create(gmsUrl); + return RestEmitter.create(gmsUrl); } log.error("GMS URL not configured."); diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java index aab25fc85e2ef5..f3cfae1f4f3634 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/spark/interceptor/LineageUtils.java @@ -15,7 +15,7 @@ import com.linkedin.data.ByteString; import com.linkedin.data.template.JacksonDataTemplateCodec; import com.linkedin.data.template.RecordTemplate; -import com.linkedin.datahub.lineage.consumer.impl.MCPEmitter; +import com.linkedin.datahub.lineage.consumer.impl.McpEmitter; import com.linkedin.datahub.lineage.spark.model.LineageConsumer; import com.linkedin.mxe.GenericAspect; @@ -38,7 +38,7 @@ public class LineageUtils { static { // system defined consumers - registerConsumer("mcpEmitter", new MCPEmitter()); + registerConsumer("mcpEmitter", new McpEmitter()); } private LineageUtils() { From a7878f67577c6db54f7eecc783e97a430ead941f Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Thu, 23 Dec 2021 08:17:20 +0530 Subject: [PATCH 14/24] simplifying serialization, using http-client, adding one test --- .../java/datahub-client/build.gradle | 5 +- .../client/DefaultHttpClientFactory.java | 14 ++ .../datahub/client/HttpClientFactory.java | 10 ++ .../client/MetadataChangeProposalWrapper.java | 31 ++++ .../main/java/datahub/client/RestEmitter.java | 135 ++++++++++-------- .../java/datahub/client/RestEmitterTest.java | 64 +++++++++ 6 files changed, 201 insertions(+), 58 deletions(-) create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java create mode 100644 metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java diff --git a/metadata-integration/java/datahub-client/build.gradle b/metadata-integration/java/datahub-client/build.gradle index 502092853d61f9..0b8dc555e1c422 100644 --- a/metadata-integration/java/datahub-client/build.gradle +++ b/metadata-integration/java/datahub-client/build.gradle @@ -1,13 +1,14 @@ - apply plugin: 'java' apply plugin: 'com.github.johnrengelman.shadow' dependencies { compile project(':metadata-models') + compile externalDependency.httpClient compile externalDependency.jacksonDataBind compileOnly externalDependency.lombok annotationProcessor externalDependency.lombok + testCompile externalDependency.mockito } test { @@ -15,7 +16,7 @@ test { } shadowJar { - zip64=true + zip64=true classifier='' } diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java new file mode 100644 index 00000000000000..350a595aeac8d7 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java @@ -0,0 +1,14 @@ +package datahub.client; + +import org.apache.http.client.HttpClient; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; + + +public class DefaultHttpClientFactory implements HttpClientFactory { + @Override + public HttpClient getHttpClient() { + CloseableHttpClient httpClient = HttpClients.createDefault(); + return httpClient; + } +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java new file mode 100644 index 00000000000000..d757732643d458 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java @@ -0,0 +1,10 @@ +package datahub.client; + +import org.apache.http.client.HttpClient; + + +public interface HttpClientFactory { + + HttpClient getHttpClient(); + +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java new file mode 100644 index 00000000000000..9885df1b784916 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java @@ -0,0 +1,31 @@ +package datahub.client; + +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; +import com.linkedin.common.urn.Urn; +import com.linkedin.data.template.DataTemplate; +import com.linkedin.events.metadata.ChangeType; +import lombok.Builder; +import lombok.Value; + + +/** + * A class that makes it easy to create new {@link MetadataChangeProposal} events + * @param + */ +@JsonDeserialize(builder = MetadataChangeProposalWrapper.MetadataChangeProposalWrapperBuilder.class) +@Value +@Builder +public class MetadataChangeProposalWrapper { + + String entityType; + Urn entityUrn; + ChangeType changeType; + T aspect; + String aspectName; + + @JsonPOJOBuilder(withPrefix = "") + public static class MetadataChangeProposalWrapperBuilder { + } + +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java index 0387752c6303d5..d3acd59fbf323b 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java @@ -1,31 +1,34 @@ package datahub.client; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.OutputStream; -import java.net.HttpURLConnection; -import java.net.URL; -import java.util.HashMap; - -import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.data.ByteString; +import com.linkedin.data.DataMap; import com.linkedin.data.template.JacksonDataTemplateCodec; +import com.linkedin.mxe.GenericAspect; import com.linkedin.mxe.MetadataChangeProposal; - +import java.io.IOException; +import java.nio.charset.StandardCharsets; import lombok.Getter; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; +import org.apache.http.HttpResponse; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.StringEntity; + @Slf4j @RequiredArgsConstructor public class RestEmitter { - private static final JacksonDataTemplateCodec DATA_TEMPLATE_CODEC = new JacksonDataTemplateCodec(); private static final int DEFAULT_CONNECT_TIMEOUT_SEC = 30; private static final int DEFAULT_READ_TIMEOUT_SEC = 30; - - + private static final String DEFAULT_AUTH_TOKEN = null; + private static final HttpClientFactory DEFAULT_HTTP_CLIENT_FACTORY = new DefaultHttpClientFactory(); + + @Getter private final String gmsUrl; @@ -38,69 +41,89 @@ public class RestEmitter { @Getter private final String token; + @Getter + private final HttpClientFactory httpClientFactory; + + private final String ingestProposalUrl; + private final String configUrl; + + private final ObjectMapper objectMapper = new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL); + private final JacksonDataTemplateCodec dataTemplateCodec = new JacksonDataTemplateCodec(objectMapper.getFactory()); + private final HttpClient httpClient; + + public RestEmitter(String serverAddr, int connectTimeoutSec, int readTimeoutSec, String token, HttpClientFactory httpClientFactory) { + this.gmsUrl = serverAddr; + this.connectTimeoutSec = connectTimeoutSec; + this.readTimeoutSec = readTimeoutSec; + this.token = token; + this.httpClientFactory = httpClientFactory; + this.httpClient = this.httpClientFactory.getHttpClient(); + this.ingestProposalUrl = this.gmsUrl + "/aspects?action=ingestProposal"; + this.configUrl = this.gmsUrl + "/config"; + } + + public void emit(MetadataChangeProposalWrapper mcpw) throws IOException { + String serializedAspect = dataTemplateCodec.dataTemplateToString(mcpw.getAspect()); + MetadataChangeProposal mcp = new MetadataChangeProposal().setEntityType(mcpw.getEntityType()) + .setAspectName(mcpw.getAspectName()) + .setEntityUrn(mcpw.getEntityUrn()) + .setChangeType(mcpw.getChangeType()) + .setAspect(new GenericAspect().setContentType("application/json") + .setValue(ByteString.unsafeWrap(serializedAspect.getBytes(StandardCharsets.UTF_8)))); + emit(mcp); + } + public void emit(MetadataChangeProposal mcp) throws IOException { - String payloadJson = DATA_TEMPLATE_CODEC.mapToString(mcp.data()); - ObjectMapper om = new ObjectMapper(); - TypeReference> typeRef = new TypeReference>() { - }; - HashMap o = om.readValue(payloadJson, typeRef); - while (o.values().remove(null)) { - } - payloadJson = om.writeValueAsString(o); - payloadJson = "{" + " \"proposal\" :" + payloadJson + "}"; - log.debug("Emitting payload: " + payloadJson + "\n to URL " + this.gmsUrl + "/aspects?action=ingestProposal"); - this.makeRequest(this.gmsUrl + "/aspects?action=ingestProposal", "POST", payloadJson); + DataMap map = new DataMap(); + map.put("proposal", mcp.data()); + String serializedMCP = dataTemplateCodec.mapToString(map); + log.debug("Emitting payload: " + serializedMCP + "\n to URL " + this.ingestProposalUrl); + this.postGeneric(this.ingestProposalUrl, serializedMCP); } - public boolean makeRequest(String urlStr, String method, String payloadJson) throws IOException { - URL url = new URL(urlStr); - HttpURLConnection con = (HttpURLConnection) url.openConnection(); - con.setConnectTimeout(this.connectTimeoutSec * 1000); - con.setReadTimeout(this.readTimeoutSec * 1000); - con.setRequestMethod(method); - con.setRequestProperty("Content-Type", "application/json"); - con.setRequestProperty("X-RestLi-Protocol-Version", "2.0.0"); - if (this.token != null) { - con.setRequestProperty("Authorization", "Bearer " + token); + private boolean postGeneric(String urlStr, String payloadJson) throws IOException { + HttpPost httpPost = new HttpPost(urlStr); + httpPost.setHeader("Content-Type", "application/json"); + httpPost.setHeader("X-RestLi-Protocol-Version", "2.0.0"); + httpPost.setHeader("Accept", "application/json"); + if (token != null) { + httpPost.setHeader("Authorization", "Bearer " + token); } - con.setDoOutput(true); - if (payloadJson != null) { - try (OutputStream os = con.getOutputStream()) { - byte[] input = payloadJson.getBytes("utf-8"); - os.write(input, 0, input.length); - } - } - try (BufferedReader br = new BufferedReader(new InputStreamReader(con.getInputStream(), "utf-8"))) { - StringBuilder response = new StringBuilder(); - String responseLine = null; - while ((responseLine = br.readLine()) != null) { - response.append(responseLine.trim()); - } - log.debug("URL: " + urlStr + " Response: " + response.toString()); - } - return true; + httpPost.setEntity(new StringEntity(payloadJson)); + HttpResponse response = httpClient.execute(httpPost); + return (response != null && response.getStatusLine() != null && response.getStatusLine().getStatusCode() == 200); + } + private boolean getGeneric(String urlStr) throws IOException { + HttpGet httpGet = new HttpGet(urlStr); + httpGet.setHeader("Content-Type", "application/json"); + httpGet.setHeader("X-RestLi-Protocol-Version", "2.0.0"); + httpGet.setHeader("Accept", "application/json"); + HttpResponse response = this.httpClient.execute(httpGet); + return (response.getStatusLine().getStatusCode() == 200); } public boolean testConnection() throws IOException { - this.makeRequest(this.gmsUrl + "/config", "GET", null); - return true; + return this.getGeneric(this.configUrl); } public static RestEmitter create(String gmsUrl) { - return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, null); + return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, null, DEFAULT_HTTP_CLIENT_FACTORY); } public static RestEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec) { - return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, null); + return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, null, DEFAULT_HTTP_CLIENT_FACTORY); } public static RestEmitter create(String gmsUrl, String token) { - return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, token); + return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, token, DEFAULT_HTTP_CLIENT_FACTORY); } public static RestEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec, String token) { - return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, token); + return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, token, DEFAULT_HTTP_CLIENT_FACTORY); } + public static RestEmitter create(String gmsUrl, HttpClientFactory httpClientFactory) { + return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, DEFAULT_AUTH_TOKEN, DEFAULT_HTTP_CLIENT_FACTORY); + } } \ No newline at end of file diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java new file mode 100644 index 00000000000000..52fbb721991343 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java @@ -0,0 +1,64 @@ +package datahub.client; + +import com.linkedin.common.urn.Urn; +import com.linkedin.dataset.DatasetProperties; +import com.linkedin.events.metadata.ChangeType; +import java.io.IOException; +import java.io.InputStream; +import java.net.URISyntaxException; +import java.nio.charset.Charset; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpPost; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnitRunner; + + +@RunWith(MockitoJUnitRunner.class) +public class RestEmitterTest { + + @Mock + HttpClient mockClient; + + @Captor + ArgumentCaptor postArgumentCaptor; + + @Test + public void testPost() throws URISyntaxException, IOException { + + HttpClientFactory mockHttpClientFactory = new HttpClientFactory() { + @Override + public HttpClient getHttpClient() { + return mockClient; + } + }; + + RestEmitter emitter = new RestEmitter("http://localhost:8080", 30, 30, null, mockHttpClientFactory); + MetadataChangeProposalWrapper mcp = + new MetadataChangeProposalWrapper.MetadataChangeProposalWrapperBuilder().entityType("dataset") + .aspectName("datasetProperties") + .changeType(ChangeType.UPSERT) + .aspect(new DatasetProperties().setDescription("Test Dataset")) + .entityUrn(Urn.createFromString("urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)")) + .build(); + emitter.emit(mcp); + Mockito.verify(mockClient).execute(postArgumentCaptor.capture()); + HttpPost testPost = postArgumentCaptor.getValue(); + Assert.assertEquals("2.0.0", testPost.getFirstHeader("X-RestLi-Protocol-Version").getValue()); + InputStream is = testPost.getEntity().getContent(); + byte[] contentBytes = new byte[(int) testPost.getEntity().getContentLength()]; + is.read(contentBytes); + String contentString = new String(contentBytes, Charset.forName("UTF-8")); + String expectedContent = + "{\"proposal\":{\"aspectName\":\"datasetProperties\"," + + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)\"," + + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset\\\"}\"}}}"; + Assert.assertEquals(expectedContent, contentString); + } +} \ No newline at end of file From 752adadf94192eb0c2d18f690deb2e6ea58fb76d Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Wed, 29 Dec 2021 08:08:09 +0530 Subject: [PATCH 15/24] refactoring interfaces, adding async and callback --- build.gradle | 3 + .../java/datahub-client/build.gradle | 4 +- .../main/java/datahub/client/Callback.java | 22 ++ .../client/DefaultHttpClientFactory.java | 14 - .../src/main/java/datahub/client/Emitter.java | 19 ++ .../datahub/client/HttpClientFactory.java | 10 - .../client/MetadataChangeProposalWrapper.java | 31 -- .../client/MetadataResponseFuture.java | 80 +++++ .../datahub/client/MetadataWriteResponse.java | 35 ++ .../main/java/datahub/client/RestEmitter.java | 192 ++++++----- .../datahub/client/RestEmitterConfig.java | 62 ++++ .../java/datahub/event/EventFormatter.java | 57 ++++ .../event/EventValidationException.java | 10 + .../event/MetadataChangeProposalWrapper.java | 98 ++++++ .../java/datahub/client/RestEmitterTest.java | 314 ++++++++++++++++-- .../datahub/event/EventFormatterTest.java | 33 ++ .../MetadataChangeProposalWrapperTest.java | 69 ++++ .../datahub/server/TestDataHubServer.java | 42 +++ .../lineage/consumer/impl/McpEmitter.java | 15 +- 19 files changed, 953 insertions(+), 157 deletions(-) create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java delete mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java delete mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java delete mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/event/EventValidationException.java create mode 100644 metadata-integration/java/datahub-client/src/main/java/datahub/event/MetadataChangeProposalWrapper.java create mode 100644 metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java create mode 100644 metadata-integration/java/datahub-client/src/test/java/datahub/event/MetadataChangeProposalWrapperTest.java create mode 100644 metadata-integration/java/datahub-client/src/test/java/datahub/server/TestDataHubServer.java diff --git a/build.gradle b/build.gradle index 3b6818b4b575a5..45d2a73d0b8863 100644 --- a/build.gradle +++ b/build.gradle @@ -72,6 +72,7 @@ project.ext.externalDependency = [ 'hadoopMapreduceClient':'org.apache.hadoop:hadoop-mapreduce-client-core:2.7.2', 'hibernateCore': 'org.hibernate:hibernate-core:5.2.16.Final', 'httpClient': 'org.apache.httpcomponents:httpclient:4.5.9', + 'httpAsyncClient': 'org.apache.httpcomponents:httpasyncclient:4.1.5', 'iStackCommons': 'com.sun.istack:istack-commons-runtime:4.0.1', 'jacksonCore': 'com.fasterxml.jackson.core:jackson-core:2.9.10', 'jacksonDataBind': 'com.fasterxml.jackson.core:jackson-databind:2.9.10.7', @@ -94,6 +95,8 @@ project.ext.externalDependency = [ 'mavenArtifact': "org.apache.maven:maven-artifact:$mavenVersion", 'mockito': 'org.mockito:mockito-core:3.0.0', 'mockitoInline': 'org.mockito:mockito-inline:3.0.0', + 'mockServer': 'org.mock-server:mockserver-netty:5.11.2', + 'mockServerClient': 'org.mock-server:mockserver-client-java:5.11.2', 'mysqlConnector': 'mysql:mysql-connector-java:8.0.20', 'neo4jHarness': 'org.neo4j.test:neo4j-harness:3.4.11', 'neo4jJavaDriver': 'org.neo4j.driver:neo4j-java-driver:4.0.1', diff --git a/metadata-integration/java/datahub-client/build.gradle b/metadata-integration/java/datahub-client/build.gradle index 0b8dc555e1c422..c133984c81176e 100644 --- a/metadata-integration/java/datahub-client/build.gradle +++ b/metadata-integration/java/datahub-client/build.gradle @@ -4,11 +4,13 @@ apply plugin: 'com.github.johnrengelman.shadow' dependencies { compile project(':metadata-models') - compile externalDependency.httpClient + compile externalDependency.httpAsyncClient compile externalDependency.jacksonDataBind compileOnly externalDependency.lombok annotationProcessor externalDependency.lombok testCompile externalDependency.mockito + testCompile externalDependency.mockServer + testCompile externalDependency.mockServerClient } test { diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java new file mode 100644 index 00000000000000..cee3f95f063857 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java @@ -0,0 +1,22 @@ +package datahub.client; + +import javax.annotation.Nullable; + + +public interface Callback { + + /** + * Called when the client request has completed. + * Completion does not imply success. Inspect the response object to understand if + * this was a successfully processed request or not. + * @param response + */ + void onCompletion(@Nullable T response); + + /** + * Called when the client request has thrown an exception before completion. + * @param exception + */ + void onFailure(Throwable exception); + +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java deleted file mode 100644 index 350a595aeac8d7..00000000000000 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/DefaultHttpClientFactory.java +++ /dev/null @@ -1,14 +0,0 @@ -package datahub.client; - -import org.apache.http.client.HttpClient; -import org.apache.http.impl.client.CloseableHttpClient; -import org.apache.http.impl.client.HttpClients; - - -public class DefaultHttpClientFactory implements HttpClientFactory { - @Override - public HttpClient getHttpClient() { - CloseableHttpClient httpClient = HttpClients.createDefault(); - return httpClient; - } -} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java new file mode 100644 index 00000000000000..3d3d359f7f1db3 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java @@ -0,0 +1,19 @@ +package datahub.client; + +import com.linkedin.mxe.MetadataChangeProposal; +import datahub.event.MetadataChangeProposalWrapper; +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + + +public interface Emitter extends Closeable { + + Future emit(MetadataChangeProposalWrapper mcpw, Callback callback) throws IOException; + + Future emit(MetadataChangeProposal mcp, Callback callback) throws IOException; + + boolean testConnection() throws IOException, ExecutionException, InterruptedException; + +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java deleted file mode 100644 index d757732643d458..00000000000000 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/HttpClientFactory.java +++ /dev/null @@ -1,10 +0,0 @@ -package datahub.client; - -import org.apache.http.client.HttpClient; - - -public interface HttpClientFactory { - - HttpClient getHttpClient(); - -} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java deleted file mode 100644 index 9885df1b784916..00000000000000 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataChangeProposalWrapper.java +++ /dev/null @@ -1,31 +0,0 @@ -package datahub.client; - -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonPOJOBuilder; -import com.linkedin.common.urn.Urn; -import com.linkedin.data.template.DataTemplate; -import com.linkedin.events.metadata.ChangeType; -import lombok.Builder; -import lombok.Value; - - -/** - * A class that makes it easy to create new {@link MetadataChangeProposal} events - * @param - */ -@JsonDeserialize(builder = MetadataChangeProposalWrapper.MetadataChangeProposalWrapperBuilder.class) -@Value -@Builder -public class MetadataChangeProposalWrapper { - - String entityType; - Urn entityUrn; - ChangeType changeType; - T aspect; - String aspectName; - - @JsonPOJOBuilder(withPrefix = "") - public static class MetadataChangeProposalWrapperBuilder { - } - -} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java new file mode 100644 index 00000000000000..8fc9acaac9697d --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java @@ -0,0 +1,80 @@ +package datahub.client; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import lombok.SneakyThrows; +import org.apache.http.HttpResponse; + + +public class MetadataResponseFuture implements Future { + private final Future requestFuture; + private final AtomicReference responseReference; + private final CountDownLatch responseLatch; + private final ResponseMapper mapper; + + @FunctionalInterface + public interface ResponseMapper { + MetadataWriteResponse map(HttpResponse httpResponse); + } + + public MetadataResponseFuture(Future underlyingFuture, AtomicReference responseAtomicReference, CountDownLatch responseLatch) { + this.requestFuture = underlyingFuture; + this.responseReference = responseAtomicReference; + this.responseLatch = responseLatch; + this.mapper = null; + } + + public MetadataResponseFuture(Future underlyingFuture, ResponseMapper mapper) { + this.requestFuture = underlyingFuture; + this.responseReference = null; + this.responseLatch = null; + this.mapper = mapper; + } + + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return requestFuture.cancel(mayInterruptIfRunning); + } + + @Override + public boolean isCancelled() { + return requestFuture.isCancelled(); + } + + @Override + public boolean isDone() { + return requestFuture.isDone(); + } + + @SneakyThrows + @Override + public MetadataWriteResponse get() throws InterruptedException, ExecutionException { + HttpResponse response = requestFuture.get(); + if (mapper != null) { + return mapper.map(response); + } else { + // We wait for the callback to fill this out + responseLatch.await(); + return responseReference.get(); + } + } + + @Override + public MetadataWriteResponse get(long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + HttpResponse response = requestFuture.get(timeout, unit); + if (mapper != null) { + return mapper.map(response); + } else { + // We wait for the callback to fill this out + responseLatch.await(); + return responseReference.get(); + } + } + +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java new file mode 100644 index 00000000000000..d0abf747a8e338 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java @@ -0,0 +1,35 @@ +package datahub.client; + +import java.util.Map; +import lombok.Builder; +import lombok.Value; + + +@Value +@Builder +public class MetadataWriteResponse { + + /** + * True if the client send succeeded and we got a successful response from the server + */ + @Builder.Default + boolean success = true; + + Map responseMetadata; + + /** + * If the write failed due to an exception thrown by the server + * and we have access to it, then we store the stack trace here + */ + String serverException; + + /** + * The underlying response object + * (typically an HTTPResponse or a kafka.ResponseMetadata) + */ + Object underlyingResponse; + + + Throwable clientException; + +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java index d3acd59fbf323b..7442e04ebde897 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java @@ -2,128 +2,168 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.databind.ObjectMapper; -import com.linkedin.data.ByteString; import com.linkedin.data.DataMap; import com.linkedin.data.template.JacksonDataTemplateCodec; -import com.linkedin.mxe.GenericAspect; import com.linkedin.mxe.MetadataChangeProposal; +import datahub.event.EventFormatter; +import datahub.event.MetadataChangeProposalWrapper; +import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import lombok.Getter; -import lombok.RequiredArgsConstructor; +import java.io.InputStream; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import lombok.extern.slf4j.Slf4j; import org.apache.http.HttpResponse; -import org.apache.http.client.HttpClient; +import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpPost; +import org.apache.http.concurrent.FutureCallback; import org.apache.http.entity.StringEntity; +import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; @Slf4j -@RequiredArgsConstructor -public class RestEmitter { - - private static final int DEFAULT_CONNECT_TIMEOUT_SEC = 30; - private static final int DEFAULT_READ_TIMEOUT_SEC = 30; - private static final String DEFAULT_AUTH_TOKEN = null; - private static final HttpClientFactory DEFAULT_HTTP_CLIENT_FACTORY = new DefaultHttpClientFactory(); - - - @Getter - private final String gmsUrl; - - @Getter - private final int connectTimeoutSec; - - @Getter - private final int readTimeoutSec; - - @Getter - private final String token; - - @Getter - private final HttpClientFactory httpClientFactory; +public class RestEmitter implements Emitter { + private final RestEmitterConfig config; private final String ingestProposalUrl; private final String configUrl; private final ObjectMapper objectMapper = new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL); private final JacksonDataTemplateCodec dataTemplateCodec = new JacksonDataTemplateCodec(objectMapper.getFactory()); - private final HttpClient httpClient; - - public RestEmitter(String serverAddr, int connectTimeoutSec, int readTimeoutSec, String token, HttpClientFactory httpClientFactory) { - this.gmsUrl = serverAddr; - this.connectTimeoutSec = connectTimeoutSec; - this.readTimeoutSec = readTimeoutSec; - this.token = token; - this.httpClientFactory = httpClientFactory; - this.httpClient = this.httpClientFactory.getHttpClient(); - this.ingestProposalUrl = this.gmsUrl + "/aspects?action=ingestProposal"; - this.configUrl = this.gmsUrl + "/config"; + private final CloseableHttpAsyncClient httpClient; + private final EventFormatter eventFormatter; + + public RestEmitter( + RestEmitterConfig config + ) { + this.config = config; + // Override httpClient settings with RestEmitter configs if present + if (config.getTimeoutSec() != null) { + HttpAsyncClientBuilder httpClientBuilder = this.config.getAsyncHttpClientBuilder(); + httpClientBuilder.setDefaultRequestConfig(RequestConfig.custom() + .setConnectTimeout(config.getTimeoutSec() * 1000) + .setSocketTimeout(config.getTimeoutSec() * 1000) + .build()); + } + this.httpClient = this.config.getAsyncHttpClientBuilder().build(); + this.httpClient.start(); + this.ingestProposalUrl = this.config.getGmsUrl() + "/aspects?action=ingestProposal"; + this.configUrl = this.config.getGmsUrl() + "/config"; + this.eventFormatter = this.config.getEventFormatter(); } - public void emit(MetadataChangeProposalWrapper mcpw) throws IOException { - String serializedAspect = dataTemplateCodec.dataTemplateToString(mcpw.getAspect()); - MetadataChangeProposal mcp = new MetadataChangeProposal().setEntityType(mcpw.getEntityType()) - .setAspectName(mcpw.getAspectName()) - .setEntityUrn(mcpw.getEntityUrn()) - .setChangeType(mcpw.getChangeType()) - .setAspect(new GenericAspect().setContentType("application/json") - .setValue(ByteString.unsafeWrap(serializedAspect.getBytes(StandardCharsets.UTF_8)))); - emit(mcp); + @Override + public Future emit(MetadataChangeProposalWrapper mcpw, Callback callback) throws IOException { + return emit(this.eventFormatter.convert(mcpw), callback); } - public void emit(MetadataChangeProposal mcp) throws IOException { + @Override + public Future emit(MetadataChangeProposal mcp, Callback callback) throws IOException { DataMap map = new DataMap(); map.put("proposal", mcp.data()); String serializedMCP = dataTemplateCodec.mapToString(map); - log.debug("Emitting payload: " + serializedMCP + "\n to URL " + this.ingestProposalUrl); - this.postGeneric(this.ingestProposalUrl, serializedMCP); + log.debug("Emit: URL: {}, Payload: {}\n", this.ingestProposalUrl, serializedMCP); + return this.postGeneric(this.ingestProposalUrl, serializedMCP, mcp, callback); } - private boolean postGeneric(String urlStr, String payloadJson) throws IOException { + private Future postGeneric(String urlStr, String payloadJson, Object originalRequest, Callback callback) throws IOException { HttpPost httpPost = new HttpPost(urlStr); httpPost.setHeader("Content-Type", "application/json"); httpPost.setHeader("X-RestLi-Protocol-Version", "2.0.0"); httpPost.setHeader("Accept", "application/json"); - if (token != null) { - httpPost.setHeader("Authorization", "Bearer " + token); + this.config.getExtraHeaders().forEach((k, v) -> httpPost.setHeader(k, v)); + if (this.config.getToken() != null) { + httpPost.setHeader("Authorization", "Bearer " + this.config.getToken()); } httpPost.setEntity(new StringEntity(payloadJson)); - HttpResponse response = httpClient.execute(httpPost); - return (response != null && response.getStatusLine() != null && response.getStatusLine().getStatusCode() == 200); + AtomicReference responseAtomicReference = new AtomicReference<>(); + CountDownLatch responseLatch = new CountDownLatch(1); + FutureCallback httpCallback = new FutureCallback() { + @Override + public void completed(HttpResponse response) { + MetadataWriteResponse writeResponse = null; + try { + writeResponse = mapResponse(response); + responseAtomicReference.set(writeResponse); + } catch (Exception e) { + // do nothing + } + responseLatch.countDown(); + if (callback != null) { + callback.onCompletion(writeResponse); + } + } + + @Override + public void failed(Exception ex) { + callback.onFailure(ex); + } + + @Override + public void cancelled() { + callback.onFailure(new RuntimeException("Cancelled")); + } + }; + Future requestFuture = httpClient.execute(httpPost, httpCallback); + return new MetadataResponseFuture(requestFuture, responseAtomicReference, responseLatch); } - private boolean getGeneric(String urlStr) throws IOException { + private static MetadataWriteResponse mapResponse(HttpResponse response) { + MetadataWriteResponse.MetadataWriteResponseBuilder builder = MetadataWriteResponse.builder() + .underlyingResponse(response); + if ((response!= null) && (response.getStatusLine()!=null) && (response.getStatusLine().getStatusCode() == 200 || response.getStatusLine().getStatusCode() == 201)) { + builder.success(true); + } else { + builder.success(false); + try { + ByteArrayOutputStream result = new ByteArrayOutputStream(); + InputStream contentStream = response.getEntity().getContent(); + byte[] buffer = new byte[1024]; + for (int length; (length = contentStream.read(buffer)) != -1; ) { + result.write(buffer, 0, length); + } + builder.serverException(result.toString("UTF-8")); + } + catch (Exception e) { + // Catch all exceptions and still return a valid response object + log.warn("Wasn't able to convert response into a string", e); + } + } + return builder.build(); + } + + + private Future getGeneric(String urlStr) throws IOException { HttpGet httpGet = new HttpGet(urlStr); httpGet.setHeader("Content-Type", "application/json"); httpGet.setHeader("X-RestLi-Protocol-Version", "2.0.0"); httpGet.setHeader("Accept", "application/json"); - HttpResponse response = this.httpClient.execute(httpGet); - return (response.getStatusLine().getStatusCode() == 200); - } - - public boolean testConnection() throws IOException { - return this.getGeneric(this.configUrl); - } - - public static RestEmitter create(String gmsUrl) { - return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, null, DEFAULT_HTTP_CLIENT_FACTORY); + Future response = this.httpClient.execute(httpGet, null); + return new MetadataResponseFuture(response, RestEmitter::mapResponse); } - public static RestEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec) { - return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, null, DEFAULT_HTTP_CLIENT_FACTORY); + @Override + public boolean testConnection() throws IOException, ExecutionException, InterruptedException { + return this.getGeneric(this.configUrl).get().isSuccess(); } - public static RestEmitter create(String gmsUrl, String token) { - return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, token, DEFAULT_HTTP_CLIENT_FACTORY); + public static RestEmitter create(Consumer builderSupplier) { + RestEmitter restEmitter = new RestEmitter(RestEmitterConfig.builder().with(builderSupplier).build()); + return restEmitter; } - public static RestEmitter create(String gmsUrl, int connectTimeoutSec, int readTimeoutSec, String token) { - return new RestEmitter(gmsUrl, connectTimeoutSec, readTimeoutSec, token, DEFAULT_HTTP_CLIENT_FACTORY); + public static RestEmitter createWithDefaults() { + // No-op creator -> creates RestEmitter using default settings + return create(b -> {}); } - public static RestEmitter create(String gmsUrl, HttpClientFactory httpClientFactory) { - return new RestEmitter(gmsUrl, DEFAULT_CONNECT_TIMEOUT_SEC, DEFAULT_READ_TIMEOUT_SEC, DEFAULT_AUTH_TOKEN, DEFAULT_HTTP_CLIENT_FACTORY); + @Override + public void close() throws IOException { + this.httpClient.close(); } } \ No newline at end of file diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java new file mode 100644 index 00000000000000..9e3d3497c2e56c --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java @@ -0,0 +1,62 @@ +package datahub.client; + +import datahub.event.EventFormatter; +import java.util.Collections; +import java.util.Map; +import java.util.function.Consumer; +import lombok.Builder; +import lombok.NonNull; +import lombok.Value; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; + + +@Value +@Builder +public class RestEmitterConfig { + + private static final int DEFAULT_CONNECT_TIMEOUT_SEC = 30; + private static final int DEFAULT_READ_TIMEOUT_SEC = 30; + private static final String DEFAULT_AUTH_TOKEN = null; + //private static final HttpClientFactory DEFAULT_HTTP_CLIENT_FACTORY = new HttpClientFactory() {}; + + @Builder.Default + private final String gmsUrl = "http://localhost:8080"; + + private final Integer timeoutSec; + + @Builder.Default + private final String token = DEFAULT_AUTH_TOKEN; + + @Builder.Default + @NonNull + private final Map extraHeaders = Collections.EMPTY_MAP; + + private final String caCertificatePath; + + private final HttpAsyncClientBuilder asyncHttpClientBuilder; + + @Builder.Default + private final EventFormatter eventFormatter = new EventFormatter(EventFormatter.Format.PEGASUS_JSON); + + public static class RestEmitterConfigBuilder { + + private HttpAsyncClientBuilder asyncHttpClientBuilder = HttpAsyncClientBuilder + .create() + .setDefaultRequestConfig(RequestConfig.custom() + .setConnectTimeout(DEFAULT_CONNECT_TIMEOUT_SEC * 1000) + .setSocketTimeout(DEFAULT_READ_TIMEOUT_SEC * 1000) + .build()); + + public RestEmitterConfigBuilder with(Consumer builderFunction) { + builderFunction.accept(this); + return this; + } + + public RestEmitterConfigBuilder customizeHttpAsyncClient( + Consumer asyncClientBuilderFunction) { + asyncClientBuilderFunction.accept(this.asyncHttpClientBuilder); + return this; + } + } +} \ No newline at end of file diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java new file mode 100644 index 00000000000000..d5fb14e3e0a606 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java @@ -0,0 +1,57 @@ +package datahub.event; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.linkedin.common.urn.Urn; +import com.linkedin.data.ByteString; +import com.linkedin.data.template.JacksonDataTemplateCodec; +import com.linkedin.mxe.GenericAspect; +import com.linkedin.mxe.MetadataChangeProposal; +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.charset.StandardCharsets; +import javax.print.URIException; +import lombok.SneakyThrows; + + +/** + * A class that helps to format Metadata events for transport + */ +public class EventFormatter { + + public enum Format { + PEGASUS_JSON, + }; + + private final ObjectMapper objectMapper = new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL); + private final JacksonDataTemplateCodec dataTemplateCodec = new JacksonDataTemplateCodec(objectMapper.getFactory()); + private final Format serializationFormat; + + public EventFormatter(Format serializationFormat) { + this.serializationFormat = serializationFormat; + } + + public EventFormatter() { + this(Format.PEGASUS_JSON); + } + + @SneakyThrows(URISyntaxException.class) + public MetadataChangeProposal convert(MetadataChangeProposalWrapper mcpw) throws IOException { + String serializedAspect = dataTemplateCodec.dataTemplateToString(mcpw.getAspect()); + MetadataChangeProposal mcp = new MetadataChangeProposal().setEntityType(mcpw.getEntityType()) + .setAspectName(mcpw.getAspectName()) + .setEntityUrn(Urn.createFromString(mcpw.getEntityUrn())) + .setChangeType(mcpw.getChangeType()); + + switch (this.serializationFormat) { + case PEGASUS_JSON: { + mcp.setAspect(new GenericAspect().setContentType("application/json") + .setValue(ByteString.unsafeWrap(serializedAspect.getBytes(StandardCharsets.UTF_8)))); + } + break; + default: throw new EventValidationException("Cannot handle serialization format " + this.serializationFormat); + } + return mcp; + } + +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventValidationException.java b/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventValidationException.java new file mode 100644 index 00000000000000..43778cb3259717 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventValidationException.java @@ -0,0 +1,10 @@ +package datahub.event; + +public class EventValidationException extends RuntimeException { + public EventValidationException(String message) { + super(message); + } + public EventValidationException(String message, Throwable t) { + super(message, t); + } +} diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/event/MetadataChangeProposalWrapper.java b/metadata-integration/java/datahub-client/src/main/java/datahub/event/MetadataChangeProposalWrapper.java new file mode 100644 index 00000000000000..b0e4f40ae01557 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/event/MetadataChangeProposalWrapper.java @@ -0,0 +1,98 @@ +package datahub.event; + +import com.linkedin.common.urn.Urn; +import com.linkedin.data.template.DataTemplate; +import com.linkedin.events.metadata.ChangeType; +import com.linkedin.mxe.MetadataChangeProposal; +import java.net.URISyntaxException; +import java.util.Map; +import java.util.function.Consumer; +import lombok.Builder; +import lombok.Data; +import lombok.NonNull; +import lombok.extern.slf4j.Slf4j; + + +/** + * A class that makes it easy to create new {@link MetadataChangeProposal} events + * @param + */ +@Data +@Builder +@Slf4j +public class MetadataChangeProposalWrapper { + + @NonNull + String entityType; + @NonNull + String entityUrn; + @Builder.Default + ChangeType changeType = ChangeType.UPSERT; + T aspect; + String aspectName; + + /** + * Validates that this class is well formed. + * Mutates the class to auto-fill + * @throws EventValidationException is the event is not valid + */ + protected static void validate(MetadataChangeProposalWrapper mcpw) throws EventValidationException { + try { + Urn.createFromString(mcpw.entityUrn); + } catch (URISyntaxException uie) { + throw new EventValidationException("Failed to parse a valid entity urn", uie); + } + + if (mcpw.getAspect() != null && mcpw.getAspectName() == null) { + // Try to guess the aspect name from the aspect + Map schemaProps = mcpw.getAspect().schema().getProperties(); + if (schemaProps != null && schemaProps.containsKey("Aspect")) { + Object aspectProps = schemaProps.get("Aspect"); + if (aspectProps != null && aspectProps instanceof Map) { + Map aspectMap = (Map) aspectProps; + String aspectName = (String) aspectMap.get("name"); + mcpw.setAspectName(aspectName); + log.debug("Inferring aspectName as {}", aspectName); + } + } + if (mcpw.getAspectName() == null) { + throw new EventValidationException("Aspect name was null and could not be inferred."); + } + } + if (mcpw.getChangeType() != ChangeType.UPSERT) { + throw new EventValidationException("Change type other than UPSERT is not supported at this time. Supplied " + mcpw.getChangeType()); + } + if (mcpw.getChangeType() == ChangeType.UPSERT && mcpw.getAspect() == null) { + throw new EventValidationException("Aspect cannot be null if ChangeType is UPSERT"); + } + + } + + + public static MetadataChangeProposalWrapper create(Consumer builderConsumer) { + return new ValidatingMCPWBuilder().with(builderConsumer).build(); + } + + + public static MetadataChangeProposalWrapperBuilder builder() { + return new ValidatingMCPWBuilder(); + } + + public static class ValidatingMCPWBuilder extends MetadataChangeProposalWrapperBuilder { + + @Override + public MetadataChangeProposalWrapper build() { + MetadataChangeProposalWrapper mcpw = super.build(); + validate(mcpw); + return mcpw; + } + + public ValidatingMCPWBuilder with(Consumer builderConsumer) { + builderConsumer.accept(this); + return this; + } + + + } + +} diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java index 52fbb721991343..8aa15d09093cf6 100644 --- a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java @@ -1,15 +1,35 @@ package datahub.client; -import com.linkedin.common.urn.Urn; +import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.dataset.DatasetProperties; import com.linkedin.events.metadata.ChangeType; +import datahub.event.MetadataChangeProposalWrapper; +import datahub.server.TestDataHubServer; import java.io.IOException; import java.io.InputStream; import java.net.URISyntaxException; import java.nio.charset.Charset; -import org.apache.http.client.HttpClient; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpPost; +import org.apache.http.concurrent.FutureCallback; +import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; @@ -17,37 +37,43 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.junit.MockitoJUnitRunner; +import org.mockserver.matchers.Times; +import org.mockserver.model.HttpRequest; +import org.mockserver.model.RequestDefinition; + +import static org.mockserver.model.HttpRequest.*; @RunWith(MockitoJUnitRunner.class) public class RestEmitterTest { @Mock - HttpClient mockClient; + HttpAsyncClientBuilder mockHttpClientFactory; + + @Mock + CloseableHttpAsyncClient mockClient; @Captor ArgumentCaptor postArgumentCaptor; + @Captor + ArgumentCaptor callbackCaptor; + + @Before + public void setupMocks() { + Mockito.when(mockHttpClientFactory.build()).thenReturn(mockClient); + } + @Test public void testPost() throws URISyntaxException, IOException { - HttpClientFactory mockHttpClientFactory = new HttpClientFactory() { - @Override - public HttpClient getHttpClient() { - return mockClient; - } - }; - - RestEmitter emitter = new RestEmitter("http://localhost:8080", 30, 30, null, mockHttpClientFactory); + RestEmitter emitter = RestEmitter.create(b -> b.asyncHttpClientBuilder(mockHttpClientFactory)); MetadataChangeProposalWrapper mcp = - new MetadataChangeProposalWrapper.MetadataChangeProposalWrapperBuilder().entityType("dataset") - .aspectName("datasetProperties") - .changeType(ChangeType.UPSERT) - .aspect(new DatasetProperties().setDescription("Test Dataset")) - .entityUrn(Urn.createFromString("urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)")) - .build(); - emitter.emit(mcp); - Mockito.verify(mockClient).execute(postArgumentCaptor.capture()); + getMetadataChangeProposalWrapper("Test Dataset", "urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)"); + emitter.emit(mcp, null); + Mockito.verify(mockClient).execute(postArgumentCaptor.capture(), callbackCaptor.capture()); + FutureCallback callback = callbackCaptor.getValue(); + Assert.assertNotNull(callback); HttpPost testPost = postArgumentCaptor.getValue(); Assert.assertEquals("2.0.0", testPost.getFirstHeader("X-RestLi-Protocol-Version").getValue()); InputStream is = testPost.getEntity().getContent(); @@ -61,4 +87,254 @@ public HttpClient getHttpClient() { + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset\\\"}\"}}}"; Assert.assertEquals(expectedContent, contentString); } + + @Test + public void testExceptions() throws URISyntaxException, IOException, ExecutionException, InterruptedException { + + RestEmitter emitter = RestEmitter.create($ -> $ + .asyncHttpClientBuilder(mockHttpClientFactory) + ); + + MetadataChangeProposalWrapper mcp = MetadataChangeProposalWrapper.create(b -> b + .entityType("dataset") + .changeType(ChangeType.UPSERT) + .aspect(new DatasetProperties().setDescription("Test Dataset")) + .entityUrn("urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)")); + + Future mockFuture = Mockito.mock(Future.class); + Mockito.when(mockClient.execute(Mockito.any(), Mockito.any())) + .thenReturn(mockFuture); + Mockito.when(mockFuture.get()) + .thenThrow(new ExecutionException("Test execution exception", null)); + try { + emitter.emit(mcp, null).get(); + Assert.fail("should not be here"); + } catch (ExecutionException e) { + Assert.assertEquals(e.getMessage(), "Test execution exception"); + } + } + + @Test + public void testExtraHeaders() throws Exception { + RestEmitter emitter = RestEmitter.create(b -> b + .asyncHttpClientBuilder(mockHttpClientFactory) + .extraHeaders(Collections.singletonMap("Test-Header", "Test-Value"))); + MetadataChangeProposalWrapper mcpw = MetadataChangeProposalWrapper.create( + b -> b.entityType("dataset") + .entityUrn("urn:li:dataset:foo") + .aspect(new DatasetProperties()) + ); + Future mockFuture = Mockito.mock(Future.class); + Mockito.when(mockClient.execute(Mockito.any(), Mockito.any())) + .thenReturn(mockFuture); + emitter.emit(mcpw, null); + Mockito.verify(mockClient).execute(postArgumentCaptor.capture(), callbackCaptor.capture()); + FutureCallback callback = callbackCaptor.getValue(); + Assert.assertNotNull(callback); + HttpPost testPost = postArgumentCaptor.getValue(); + // old headers are not modified + Assert.assertEquals("2.0.0", testPost.getFirstHeader("X-RestLi-Protocol-Version").getValue()); + // new headers are added + Assert.assertEquals("Test-Value", testPost.getFirstHeader("Test-Header").getValue()); + } + + @Test + public void mockServerTest() throws InterruptedException, ExecutionException, IOException { + TestDataHubServer testDataHubServer = new TestDataHubServer(); + Integer port = testDataHubServer.getMockServer().getPort(); + RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + Assert.assertTrue(emitter.testConnection()); + } + + @Test + public void multithreadedTestExecutors() throws Exception { + TestDataHubServer testDataHubServer = new TestDataHubServer(); + Integer port = testDataHubServer.getMockServer().getPort(); + RestEmitter emitter = RestEmitter.create(b -> b + .gmsUrl("http://localhost:" + port)); + + testDataHubServer.getMockServer() + .when( + request() + .withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withHeader("Content-type", "application/json"), + Times.unlimited()) + .respond(org.mockserver.model.HttpResponse.response() + .withStatusCode(200)); + ExecutorService executor = Executors.newFixedThreadPool(10); + ArrayList results = new ArrayList(); + Random random = new Random(); + int testIteration = random.nextInt(); + int numRequests = 100; + for (int i=0; i { + try { + Thread.sleep(random.nextInt(100)); + MetadataChangeProposalWrapper mcp = + getMetadataChangeProposalWrapper(String.format("Test Dataset %d", testIteration), + String.format("urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar-%d,PROD)", finalI)); + Future future = emitter.emit(mcp, null); + MetadataWriteResponse response = future.get(); + Assert.assertTrue(response.isSuccess()); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + })); + } + results.forEach(x -> { + try { + x.get(); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + }); + RequestDefinition[] recordedRequests = testDataHubServer.getMockServer() + .retrieveRecordedRequests( + request() + .withPath("/aspects") + .withMethod("POST") + ); + Assert.assertEquals(100, recordedRequests.length); + List requests = Arrays.stream(recordedRequests).sequential() + .filter(x -> x instanceof HttpRequest) + .map(x -> (HttpRequest) x) + .collect(Collectors.toList()); + ObjectMapper mapper = new ObjectMapper(); + for (int i = 0; i < numRequests; ++i) { + String expectedContent = String.format("{\"proposal\":{\"aspectName\":\"datasetProperties\"," + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar-%d,PROD)\"," + + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset %d\\\"}\"}}}", i, testIteration); + + Assert.assertEquals(requests.stream().filter(x -> { + String bodyString = ""; + try { + bodyString = mapper.writeValueAsString(mapper.readValue(x.getBodyAsString().getBytes(StandardCharsets.UTF_8), Map.class)); + } catch (IOException ioException) { + return false; + } + return bodyString.equals(expectedContent); + }).count(), 1); + } + + } + + private MetadataChangeProposalWrapper getMetadataChangeProposalWrapper(String description, String entityUrn) { + return MetadataChangeProposalWrapper.builder() + .entityType("dataset") + .changeType(ChangeType.UPSERT) + .aspect(new DatasetProperties().setDescription(description)) + .entityUrn(entityUrn) + .build(); + } + + @Test + public void multithreadedTestSingleThreadCaller() throws Exception { + TestDataHubServer testDataHubServer = new TestDataHubServer(); + Integer port = testDataHubServer.getMockServer().getPort(); + RestEmitter emitter = RestEmitter.create(b -> b + .gmsUrl("http://localhost:" + port)); + + testDataHubServer.getMockServer() + .when( + request() + .withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withHeader("Content-type", "application/json"), + Times.unlimited()) + .respond(org.mockserver.model.HttpResponse.response() + .withStatusCode(200)); + ArrayList results = new ArrayList(); + Random random = new Random(); + int testIteration = random.nextInt(); + int numRequests = 100; + for (int i=0; i future = emitter.emit(mcp, null); + results.add(future); + } + results.forEach(x -> { + try { + x.get(); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + }); + RequestDefinition[] recordedRequests = testDataHubServer.getMockServer() + .retrieveRecordedRequests( + request() + .withPath("/aspects") + .withMethod("POST") + ); + Assert.assertEquals(numRequests, recordedRequests.length); + List requests = Arrays.stream(recordedRequests).sequential() + .filter(x -> x instanceof HttpRequest) + .map(x -> (HttpRequest) x) + .collect(Collectors.toList()); + ObjectMapper mapper = new ObjectMapper(); + for (int i = 0; i < numRequests; ++i) { + String expectedContent = String.format("{\"proposal\":{\"aspectName\":\"datasetProperties\"," + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar-%d,PROD)\"," + + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset %d\\\"}\"}}}", i, testIteration); + + Assert.assertEquals(requests.stream().filter(x -> { + String bodyString = ""; + try { + bodyString = mapper.writeValueAsString(mapper.readValue(x.getBodyAsString().getBytes(StandardCharsets.UTF_8), Map.class)); + } catch (IOException ioException) { + return false; + } + return bodyString.equals(expectedContent); + }).count(), 1); + } + + } + + + @Test + public void testCallback() throws Exception { + TestDataHubServer testDataHubServer = new TestDataHubServer(); + Integer port = testDataHubServer.getMockServer().getPort(); + RestEmitter emitter = RestEmitter.create(b -> b + .gmsUrl("http://localhost:" + port)); + + testDataHubServer.getMockServer() + .when( + request() + .withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withHeader("Content-type", "application/json"), + Times.unlimited()) + .respond(org.mockserver.model.HttpResponse.response() + .withStatusCode(500) + .withBody("exception")); + + MetadataChangeProposalWrapper mcpw = getMetadataChangeProposalWrapper("Test Dataset", "urn:li:dataset:foo"); + AtomicReference callbackResponse = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + Future future = emitter.emit(mcpw, new Callback() { + @Override + public void onCompletion(MetadataWriteResponse response) { + callbackResponse.set(response); + Assert.assertFalse(response.isSuccess()); + latch.countDown(); + } + + @Override + public void onFailure(Throwable exception) { + Assert.fail("Should not be called"); + latch.countDown(); + } + }); + + latch.await(); + Assert.assertEquals(callbackResponse.get(), future.get()); + + + } + } \ No newline at end of file diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java new file mode 100644 index 00000000000000..aee662251ad16c --- /dev/null +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java @@ -0,0 +1,33 @@ +package datahub.event; + +import com.linkedin.common.urn.Urn; +import com.linkedin.dataset.DatasetProperties; +import com.linkedin.events.metadata.ChangeType; +import com.linkedin.mxe.MetadataChangeProposal; +import java.io.IOException; +import java.net.URISyntaxException; +import java.nio.charset.Charset; +import org.junit.Test; +import org.testng.Assert; + + +public class EventFormatterTest { + + @Test + public void testPartialMCPW() throws URISyntaxException, IOException, EventValidationException { + MetadataChangeProposalWrapper metadataChangeProposalWrapper = + MetadataChangeProposalWrapper.builder() + .entityType("dataset") + .entityUrn("urn:li:foo") + .aspect(new DatasetProperties() + .setDescription("A test dataset")) + .build(); + MetadataChangeProposalWrapper.validate(metadataChangeProposalWrapper); + EventFormatter eventFormatter = new EventFormatter(); + MetadataChangeProposal mcp = eventFormatter.convert(metadataChangeProposalWrapper); + Assert.assertEquals(mcp.getAspect().getContentType(), "application/json"); + String content = mcp.getAspect().getValue().asString(Charset.forName("UTF-8")); + Assert.assertEquals(content, "{\"description\":\"A test dataset\"}"); + } + +} diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/event/MetadataChangeProposalWrapperTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/event/MetadataChangeProposalWrapperTest.java new file mode 100644 index 00000000000000..a61a2a1380d3b3 --- /dev/null +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/event/MetadataChangeProposalWrapperTest.java @@ -0,0 +1,69 @@ +package datahub.event; + +import com.linkedin.common.AuditStamp; +import com.linkedin.common.urn.Urn; +import com.linkedin.dataset.DatasetProperties; +import java.net.URISyntaxException; +import org.junit.Assert; +import org.junit.Test; + + +public class MetadataChangeProposalWrapperTest { + + /** + * We should throw errors on validation as exceptions + */ + @Test + public void testBuilderExceptions() { + try { + MetadataChangeProposalWrapper.create(b -> b + .entityUrn("foo") // bad urn should throw exception + .entityType("dataset")); + Assert.fail("Should throw an exception"); + } catch (EventValidationException e) { + Assert.assertTrue("Underlying exception should be a URI syntax issue", e.getCause() instanceof URISyntaxException); + } catch (Exception e) { + Assert.fail("Should not throw any other exception"); + } + } + + @Test + public void testAspectInferenceSuccess() throws EventValidationException { + MetadataChangeProposalWrapper mcpw = MetadataChangeProposalWrapper.create(b -> b + .entityType("dataset") + .entityUrn("urn:li:dataset:(foo,bar,PROD)") + .aspect(new DatasetProperties())); + Assert.assertEquals(mcpw.aspectName, "datasetProperties"); + } + + /** + * We throw exceptions on using the regular builder pattern + * @throws URISyntaxException + * @throws EventValidationException + */ + @Test(expected = EventValidationException.class) + public void testAspectInferenceFailure() throws URISyntaxException, EventValidationException { + MetadataChangeProposalWrapper mcpw = MetadataChangeProposalWrapper.builder() + .entityType("dataset") + .entityUrn("urn:li:dataset:(foo,bar,PROD)") + .aspect(new AuditStamp().setActor(Urn.createFromString("urn:li:corpUser:jdoe"))) + .build(); + } + + /** + * We throw exceptions on using the lambda builder pattern + * @throws URISyntaxException + * @throws EventValidationException + */ + @Test(expected = EventValidationException.class) + public void testAspectInferenceFailureLambda() throws URISyntaxException, EventValidationException { + Urn actorUrn = Urn.createFromString("urn:li:corpUser:jdoe"); + MetadataChangeProposalWrapper mcpw = MetadataChangeProposalWrapper.create(b -> b + .entityType("dataset") + .entityUrn("urn:li:dataset:(foo,bar,PROD)") + .aspect(new AuditStamp().setActor(actorUrn)) + ); + } + + +} diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/server/TestDataHubServer.java b/metadata-integration/java/datahub-client/src/test/java/datahub/server/TestDataHubServer.java new file mode 100644 index 00000000000000..e7cdee3f369e1c --- /dev/null +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/server/TestDataHubServer.java @@ -0,0 +1,42 @@ +package datahub.server; + +import org.mockserver.integration.ClientAndServer; +import org.mockserver.matchers.Times; + +import static org.mockserver.integration.ClientAndServer.startClientAndServer; +import static org.mockserver.model.HttpRequest.*; + + +public class TestDataHubServer { + + private ClientAndServer mockServer; + + public Integer getPort() { + return mockServer.getPort(); + } + + public ClientAndServer getMockServer() { + return mockServer; + } + + public TestDataHubServer() { + mockServer = startClientAndServer(); + init(); + } + + public void init() { + mockServer + .when( + request() + .withMethod("GET") + .withPath("/config") + .withHeader("Content-type", "application/json"), + Times.unlimited() + ).respond( + org.mockserver.model.HttpResponse.response() + .withBody("{\"noCode\": true }") + ); + } + + +} diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java index ed057183562513..4e24cf55f8075a 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java @@ -1,11 +1,13 @@ package com.linkedin.datahub.lineage.consumer.impl; +import datahub.client.Emitter; import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; import java.util.List; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; import org.apache.spark.SparkConf; import org.apache.spark.SparkEnv; @@ -25,13 +27,13 @@ public class McpEmitter implements LineageConsumer { private ConcurrentHashMap singleton = new ConcurrentHashMap<>(); private void emit(List mcps) { - RestEmitter emitter = emitter(); + Emitter emitter = emitter(); if (emitter != null) { mcps.forEach(mcp -> { - log.debug("Emitting \n" + mcp); + log.debug("Emitting {}", mcp); try { - emitter.emit(mcp); - } catch (IOException e) { + emitter.emit(mcp,null).get(); + } catch (IOException | InterruptedException | ExecutionException e) { // log error, but don't impact thread StringWriter s = new StringWriter(); PrintWriter p = new PrintWriter(s); @@ -45,13 +47,14 @@ private void emit(List mcps) { // TODO ideally the impl here should not be tied to Spark; the LineageConsumer // API needs tweaking to include configs - private RestEmitter emitter() { + private Emitter emitter() { singleton.computeIfAbsent(SENTINEL, x -> { SparkConf conf = SparkEnv.get().conf(); if (conf.contains(GMS_URL_KEY)) { String gmsUrl = conf.get(GMS_URL_KEY); log.debug("REST emitter configured with GMS url " + gmsUrl); - return RestEmitter.create(gmsUrl); + return RestEmitter.create($ -> $ + .gmsUrl(gmsUrl)); } log.error("GMS URL not configured."); From b6d350b655d5aaf0d23ffb94ec81ab1a04a9944c Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Wed, 29 Dec 2021 09:10:56 +0530 Subject: [PATCH 16/24] fix checkstyle --- .../com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java index 4e24cf55f8075a..9cfeedc9c9a83f 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java @@ -32,7 +32,7 @@ private void emit(List mcps) { mcps.forEach(mcp -> { log.debug("Emitting {}", mcp); try { - emitter.emit(mcp,null).get(); + emitter.emit(mcp, null).get(); } catch (IOException | InterruptedException | ExecutionException e) { // log error, but don't impact thread StringWriter s = new StringWriter(); From 62e3e4484f94beca2e534795ca09127f691c62e4 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Wed, 29 Dec 2021 22:23:48 +0530 Subject: [PATCH 17/24] simplify tests. move from wiremock to mockserver --- spark-lineage/build.gradle | 5 +- .../datahub/lineage/TestSparkJobsLineage.java | 486 +++++++----------- 2 files changed, 200 insertions(+), 291 deletions(-) diff --git a/spark-lineage/build.gradle b/spark-lineage/build.gradle index a641d190bb4f56..4815e887920883 100644 --- a/spark-lineage/build.gradle +++ b/spark-lineage/build.gradle @@ -40,7 +40,10 @@ dependencies { testImplementation externalDependency.mockito - testImplementation(externalDependency.wiremock){ + testImplementation(externalDependency.mockServer){ + exclude group: "com.fasterxml.jackson.core" + } // older version to allow older guava + testImplementation(externalDependency.mockServerClient){ exclude group: "com.fasterxml.jackson.core" } // older version to allow older guava diff --git a/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java b/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java index fa908b587731a0..d66ebb38acb84b 100644 --- a/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java +++ b/spark-lineage/src/test/java/com/linkedin/datahub/lineage/TestSparkJobsLineage.java @@ -1,33 +1,31 @@ package com.linkedin.datahub.lineage; -import static com.github.tomakehurst.wiremock.client.WireMock.ok; -import static com.github.tomakehurst.wiremock.client.WireMock.post; -import static com.github.tomakehurst.wiremock.client.WireMock.postRequestedFor; -import static com.github.tomakehurst.wiremock.client.WireMock.urlEqualTo; -import static com.github.tomakehurst.wiremock.client.WireMock.verify; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - +import com.linkedin.datahub.lineage.spark.interceptor.LineageUtils; +import com.linkedin.datahub.lineage.spark.model.DatasetLineage; +import com.linkedin.datahub.lineage.spark.model.LineageConsumer; +import com.linkedin.datahub.lineage.spark.model.LineageEvent; +import com.linkedin.datahub.lineage.spark.model.SQLQueryExecStartEvent; +import com.linkedin.datahub.lineage.spark.model.dataset.CatalogTableDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.HdfsPathDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.JdbcDataset; +import com.linkedin.datahub.lineage.spark.model.dataset.SparkDataset; import java.io.File; import java.io.IOException; import java.nio.file.Files; -import java.nio.file.Path; import java.nio.file.Paths; -import java.nio.file.StandardOpenOption; import java.sql.Connection; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Properties; import java.util.Set; import java.util.stream.Collectors; - import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +import org.junit.After; import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; @@ -37,34 +35,23 @@ import org.junit.rules.TestRule; import org.junit.rules.TestWatcher; import org.junit.runner.Description; +import org.mockserver.integration.ClientAndServer; +import org.mockserver.matchers.Times; +import org.mockserver.model.HttpResponse; +import org.mockserver.model.JsonBody; +import org.mockserver.verify.VerificationTimes; import org.testcontainers.containers.PostgreSQLContainer; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.github.tomakehurst.wiremock.WireMockServer; -import com.github.tomakehurst.wiremock.client.MappingBuilder; -import com.github.tomakehurst.wiremock.client.WireMock; -import com.github.tomakehurst.wiremock.core.Admin; -import com.github.tomakehurst.wiremock.core.WireMockConfiguration; -import com.github.tomakehurst.wiremock.extension.Parameters; -import com.github.tomakehurst.wiremock.extension.PostServeAction; -import com.github.tomakehurst.wiremock.matching.MatchResult; -import com.github.tomakehurst.wiremock.matching.StringValuePattern; -import com.github.tomakehurst.wiremock.stubbing.ServeEvent; -import com.linkedin.datahub.lineage.spark.interceptor.LineageUtils; -import com.linkedin.datahub.lineage.spark.model.DatasetLineage; -import com.linkedin.datahub.lineage.spark.model.LineageConsumer; -import com.linkedin.datahub.lineage.spark.model.LineageEvent; -import com.linkedin.datahub.lineage.spark.model.SQLQueryExecStartEvent; -import com.linkedin.datahub.lineage.spark.model.dataset.CatalogTableDataset; -import com.linkedin.datahub.lineage.spark.model.dataset.HdfsPathDataset; -import com.linkedin.datahub.lineage.spark.model.dataset.JdbcDataset; -import com.linkedin.datahub.lineage.spark.model.dataset.SparkDataset; - +import static org.junit.Assert.*; +import static org.mockserver.integration.ClientAndServer.*; +import static org.mockserver.model.HttpRequest.*; public class TestSparkJobsLineage { - private static final boolean MOCK_GMS = Boolean.valueOf("true"); // if false, MCPs get written to real GMS server (see GMS_PORT) - private static final boolean VERIFY_EXPECTED = MOCK_GMS && Boolean.valueOf("true"); // if false, "expected" JSONs are overwritten. + private static final boolean MOCK_GMS = Boolean.valueOf("true"); + // if false, MCPs get written to real GMS server (see GMS_PORT) + private static final boolean VERIFY_EXPECTED = MOCK_GMS && Boolean.valueOf("true"); + // if false, "expected" JSONs are overwritten. private static final String APP_NAME = "sparkTestApp"; @@ -80,189 +67,82 @@ public class TestSparkJobsLineage { private static final int GMS_PORT = MOCK_GMS ? 8089 : 8080; private static final String EXPECTED_JSON_ROOT = "src/test/resources/expected/"; - + @ClassRule + public static PostgreSQLContainer db = + new PostgreSQLContainer<>("postgres:9.6.12").withDatabaseName("sparktestdb"); private static SparkSession spark; private static Properties jdbcConnnProperties; private static DatasetLineageAccumulator acc; - - @SuppressWarnings("rawtypes") - private static final class McpContentPattern extends StringValuePattern { - - public McpContentPattern(String expectedValue) { - super(expectedValue); - } - - // dataflow case, we do not match against expected string - public McpContentPattern() { - super(""); - } + private static ClientAndServer mockServer; + @Rule + public TestRule mockServerWatcher = new TestWatcher() { @Override - public MatchResult match(String actual) { - if (actual.contains("dataJobInputOutput")) { - return expectedValue.contains(relPaths(actual)) ? MatchResult.exactMatch() : MatchResult.noMatch(); - } - try { - HashMap body = new ObjectMapper().readValue(actual, HashMap.class); - HashMap proposal = (HashMap) body.get("proposal"); - String aspectName = (String) proposal.get("aspectName"); - if (aspectName.equals("dataFlowInfo")) { - return checkFlowInfo(proposal); - } - if (actual.contains("dataJobInfo")) { - return checkJobInfo(proposal); - } - return MatchResult.noMatch(); - } catch (Exception e) { - throw new RuntimeException(e); + protected void finished(Description description) { + if (!VERIFY_EXPECTED) { + return; } + verifyTestScenario(description.getMethodName()); + clear(); + super.finished(description); } + }; - private MatchResult checkJobInfo(HashMap proposal) { - // TODO check custom props etc. - return MatchResult.exactMatch(); - } - - private MatchResult checkFlowInfo(HashMap proposal) { - // TODO check custom props etc. - return MatchResult.exactMatch(); - } - + private static String addLocalPath(String s) { + return s.replaceAll("file:/" + RESOURCE_DIR, "file:" + Paths.get(RESOURCE_DIR).toAbsolutePath().toString()); } - private static String relPaths(String s) { - return s.replaceAll("file:[0-9|a-z|A-Z|\\-|\\/|_|\\.]*" + RESOURCE_DIR, "file:/" + RESOURCE_DIR); + public static void resetBaseExpectations() { + mockServer.when(request().withMethod("GET").withPath("/config").withHeader("Content-type", "application/json"), + Times.unlimited()).respond(org.mockserver.model.HttpResponse.response().withBody("{\"noCode\": true }")); + mockServer.when( + request().withMethod("POST").withPath("/aspects").withQueryStringParameter("action", "ingestProposal"), + Times.unlimited()).respond(HttpResponse.response().withStatusCode(200)); } - private static final class RequestFileWriter extends PostServeAction { - - @Override - public String getName() { - return "writeReqJson"; - } - - @SuppressWarnings({ "rawtypes", "unused" }) - public void doAction(ServeEvent serveEvent, Admin admin, Parameters parameters) { - String currentTestRes = parameters.getString("filename"); - if (currentTestRes == null || VERIFY_EXPECTED) { - return; - } - - Path p = Paths.get(EXPECTED_JSON_ROOT, currentTestRes); - System.out.println("Writing json to file " + p); - String json = serveEvent.getRequest().getBodyAsString(); - try { - HashMap body = new ObjectMapper().readValue(json, HashMap.class); - HashMap proposal = (HashMap) body.get("proposal"); - String aspectName = (String) proposal.get("aspectName"); - - // this effectively checks URNs and lineages; other jobInfo/flowInfo are checked - // in McpContentPattern - if (aspectName.equals("dataJobInputOutput")) { - json = relPaths(json); - Files.write(p, Collections.singletonList(json), StandardOpenOption.CREATE, StandardOpenOption.APPEND); - } - } catch (Exception e) { - throw new RuntimeException(e); - } - } + public static void init() { + mockServer = startClientAndServer(); + resetBaseExpectations(); } - private static class DatasetLineageAccumulator implements LineageConsumer { - - private final List lineages = new ArrayList<>(); - - public void flushJobs() { - lineages.clear(); - } - - public List getLineages() { - return Collections.unmodifiableList(lineages); - } - - @Override - public void accept(LineageEvent e) { - if (e instanceof SQLQueryExecStartEvent) { - lineages.add(((SQLQueryExecStartEvent) e).getDatasetLineage()); + public static void verifyTestScenario(String testName) { + String expectationFileName = testName + ".json"; + try { + List expected = Files.readAllLines(Paths.get(EXPECTED_JSON_ROOT, expectationFileName)); + for (String content : expected) { + String swappedContent = addLocalPath(content); + mockServer.verify(request().withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withBody(new JsonBody(swappedContent)), VerificationTimes.atLeast(1)); } + } catch (IOException ioe) { + throw new RuntimeException("failed to read expectations", ioe); } } - @Rule - public TestRule watcher = new TestWatcher() { - protected void starting(Description description) { - if (!MOCK_GMS) { - return; - } - String currentTestRes = description.getMethodName() + ".json"; - - MappingBuilder mapping = baseMapping(); - - try { - if (VERIFY_EXPECTED) { - // setup the request body that we expect - List expected = Files.readAllLines(Paths.get(EXPECTED_JSON_ROOT, currentTestRes)); - mapping.withRequestBody(new McpContentPattern(expected.toString())); - } else { - // overwrite "expected" json file with the MCP request bodies - mapping.withPostServeAction("writeReqJson", Parameters.one("filename", currentTestRes)); - Files.deleteIfExists(Paths.get(EXPECTED_JSON_ROOT, currentTestRes)); - } - } catch (IOException e) { - e.printStackTrace(); - } - wireMock.stubFor(mapping); + public static void verify(int numRequests) { + if (!VERIFY_EXPECTED) { + return; } - - @Override - protected void finished(Description description) { - if (!VERIFY_EXPECTED) { - return; - } - assertTrue(WireMock.findUnmatchedRequests().isEmpty()); - wireMock.resetRequests(); - wireMock.resetMappings(); - super.finished(description); - } - }; - - public static WireMockServer wireMock = new WireMockServer(WireMockConfiguration - .options() - .port(8089) - .extensions(new RequestFileWriter())); - - @ClassRule - public static PostgreSQLContainer db = new PostgreSQLContainer<>("postgres:9.6.12") - .withDatabaseName("sparktestdb"); - - private static MappingBuilder baseMapping() { - return post("/aspects?action=ingestProposal") - .willReturn(ok() - .withBody("SUCCESS")); + mockServer.verify( + request().withMethod("POST").withPath("/aspects").withQueryStringParameter("action", "ingestProposal"), + VerificationTimes.exactly(numRequests)); } @BeforeClass public static void setup() { - + acc = new DatasetLineageAccumulator(); LineageUtils.registerConsumer("accumulator", acc); + init(); - wireMock.start(); - WireMock.configureFor("localhost", 8089); - MappingBuilder mapping = baseMapping(); - if (VERIFY_EXPECTED) { - mapping.withRequestBody(new McpContentPattern()); - } - wireMock.stubFor(mapping); - - spark = SparkSession - .builder() + spark = SparkSession.builder() .appName(APP_NAME) .config("spark.master", MASTER) - .config("spark.extraListeners", - "com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter") + .config("spark.extraListeners", "com.linkedin.datahub.lineage.spark.interceptor.DatahubLineageEmitter") .config("spark.datahub.lineage.consumerTypes", "accumulator, mcpEmitter") - .config("spark.datahub.rest.server", "http://localhost:" + GMS_PORT) + .config("spark.datahub.rest.server", "http://localhost:" + mockServer.getPort()) .config("spark.sql.warehouse.dir", new File(WAREHOUSE_LOC).getAbsolutePath()) .enableHiveSupport() .getOrCreate(); @@ -274,28 +154,67 @@ public static void setup() { jdbcConnnProperties.put("password", db.getPassword()); if (VERIFY_EXPECTED) { - verify(1, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); - assertTrue(WireMock.findUnmatchedRequests().isEmpty()); + verify(1); + clear(); } - wireMock.resetRequests(); - wireMock.resetMappings(); + } + + private static void clear() { + mockServer.clear( + request().withMethod("POST").withPath("/aspects").withQueryStringParameter("action", "ingestProposal")); } @AfterClass public static void tearDown() throws Exception { - wireMock.resetRequests(); - wireMock.resetMappings(); - MappingBuilder mapping = baseMapping(); - if (VERIFY_EXPECTED) { - mapping.withRequestBody(new McpContentPattern()); - } - wireMock.stubFor(mapping); spark.stop(); - if (VERIFY_EXPECTED) { - verify(1, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); - assertTrue(WireMock.findUnmatchedRequests().isEmpty()); + mockServer.stop(); + } + + private static void check(List expected, List actual) { + assertEquals(expected.size(), actual.size()); + for (int i = 0; i < expected.size(); i++) { + check(expected.get(i), actual.get(i)); } - wireMock.stop(); + } + + private static void check(DatasetLineage expected, DatasetLineage actual) { + assertEquals(expected.getSink().toString(), actual.getSink().toString()); + assertEquals(dsToStrings(expected.getSources()), dsToStrings(actual.getSources())); + assertTrue(actual.getCallSiteShort().contains("TestSparkJobsLineage")); + } + + private static Set dsToStrings(Set datasets) { + return datasets.stream().map(x -> x.toString()).collect(Collectors.toSet()); + } + + private static DatasetLineage dsl(SparkDataset sink, SparkDataset... source) { + return dsl(null, sink, source); + } + + private static DatasetLineage dsl(String callSite, SparkDataset sink, SparkDataset... source) { + DatasetLineage lineage = new DatasetLineage(callSite, "unknownPlan", sink); + Arrays.asList(source).forEach(x -> lineage.addSource(x)); + return lineage; + } + + private static HdfsPathDataset hdfsDs(String fileName) { + return new HdfsPathDataset("file:" + abs(DATA_DIR + "/" + fileName)); + } + + private static JdbcDataset pgDs(String tbl) { + return new JdbcDataset(db.getJdbcUrl(), tbl); + } + + private static CatalogTableDataset catTblDs(String tbl) { + return new CatalogTableDataset(tbl(tbl)); + } + + private static String tbl(String tbl) { + return TEST_DB + "." + tbl; + } + + private static String abs(String relPath) { + return new File(relPath).getAbsolutePath(); } @Before @@ -303,47 +222,54 @@ public void before() { acc.flushJobs(); } + @After + public void after() { + resetBaseExpectations(); + } + @Test public void testHdfsInOut() throws Exception { - + Dataset df1 = spark.read().option("header", "true").csv(DATA_DIR + "/in1.csv"); Dataset df2 = spark.read().option("header", "true").csv(DATA_DIR + "/in2.csv"); df1.createOrReplaceTempView("v1"); df2.createOrReplaceTempView("v2"); - Dataset df = spark - .sql("select v1.c1 as a, v1.c2 as b, v2.c1 as c, v2.c2 as d from v1 join v2 on v1.id = v2.id"); + Dataset df = + spark.sql("select v1.c1 as a, v1.c2 as b, v2.c1 as c, v2.c2 as d from v1 join v2 on v1.id = v2.id"); // InsertIntoHadoopFsRelationCommand df.write().mode(SaveMode.Overwrite).csv(DATA_DIR + "/out.csv"); Thread.sleep(5000); check(dsl(hdfsDs("out.csv"), hdfsDs("in1.csv"), hdfsDs("in2.csv")), acc.getLineages().get(0)); if (VERIFY_EXPECTED) { - verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + verify(1 * N); } } @Test public void testHdfsInJdbcOut() throws Exception { Dataset df1 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in1.csv") - .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b"); + .option("header", "true") + .csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a") + .withColumnRenamed("c2", "b"); Dataset df2 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in2.csv") - .withColumnRenamed("c1", "c").withColumnRenamed("c2", "d"); + .option("header", "true") + .csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "c") + .withColumnRenamed("c2", "d"); Dataset df = df1.join(df2, "id").drop("id"); // SaveIntoDataSourceCommand // HadoopFsRelation input - df.write().mode(SaveMode.Overwrite).jdbc( - db.getJdbcUrl(), - "foo1", jdbcConnnProperties); + df.write().mode(SaveMode.Overwrite).jdbc(db.getJdbcUrl(), "foo1", jdbcConnnProperties); Thread.sleep(5000); check(dsl(pgDs("foo1"), hdfsDs("in1.csv"), hdfsDs("in2.csv")), acc.getLineages().get(0)); - if (VERIFY_EXPECTED) { - verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + { + verify(1 * N); } } @@ -355,35 +281,38 @@ public void testHdfsJdbcInJdbcOut() throws Exception { c.close(); Dataset df1 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in1.csv") - .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b2"); + .option("header", "true") + .csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a") + .withColumnRenamed("c2", "b2"); - Dataset df2 = spark.read() - .jdbc(db.getJdbcUrl(), "foo2", jdbcConnnProperties); + Dataset df2 = spark.read().jdbc(db.getJdbcUrl(), "foo2", jdbcConnnProperties); Dataset df = df1.join(df2, "a"); // SaveIntoDataSourceCommand // JDBCRelation input - df.write().mode(SaveMode.Overwrite).jdbc( - db.getJdbcUrl(), - "foo3", jdbcConnnProperties); + df.write().mode(SaveMode.Overwrite).jdbc(db.getJdbcUrl(), "foo3", jdbcConnnProperties); Thread.sleep(5000); check(dsl(pgDs("foo3"), hdfsDs("in1.csv"), pgDs("foo2")), acc.getLineages().get(0)); if (VERIFY_EXPECTED) { - verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + verify(1 * N); } } @Test public void testHdfsInHiveOut() throws Exception { Dataset df1 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in1.csv") - .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b"); + .option("header", "true") + .csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a") + .withColumnRenamed("c2", "b"); Dataset df2 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in2.csv") - .withColumnRenamed("c1", "c").withColumnRenamed("c2", "d"); + .option("header", "true") + .csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "c") + .withColumnRenamed("c2", "d"); Dataset df = df1.join(df2, "id").drop("id"); @@ -397,32 +326,35 @@ public void testHdfsInHiveOut() throws Exception { DatasetLineage exp = dsl(catTblDs("foo4"), hdfsDs("in1.csv"), hdfsDs("in2.csv")); check(Collections.nCopies(3, exp), acc.getLineages()); if (VERIFY_EXPECTED) { - verify(3 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + verify(3 * N); } } @Test public void testHiveInHiveOut() throws Exception { Dataset df1 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in1.csv") - .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b"); + .option("header", "true") + .csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a") + .withColumnRenamed("c2", "b"); Dataset df2 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in2.csv") - .withColumnRenamed("c1", "c").withColumnRenamed("c2", "d"); + .option("header", "true") + .csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "c") + .withColumnRenamed("c2", "d"); df1.createOrReplaceTempView("v1"); df2.createOrReplaceTempView("v2"); // CreateHiveTableAsSelectCommand - spark.sql("create table " + tbl("foo5") + " as " - + "(select v1.a, v1.b, v2.c, v2.d from v1 join v2 on v1.id = v2.id)"); + spark.sql( + "create table " + tbl("foo5") + " as " + "(select v1.a, v1.b, v2.c, v2.d from v1 join v2 on v1.id = v2.id)"); check(dsl(catTblDs("foo5"), hdfsDs("in1.csv"), hdfsDs("in2.csv")), acc.getLineages().get(0)); // CreateHiveTableAsSelectCommand - spark.sql("create table " + tbl("hivetab") + " as " - + "(select * from " + tbl("foo5") + ")"); + spark.sql("create table " + tbl("hivetab") + " as " + "(select * from " + tbl("foo5") + ")"); check(dsl(catTblDs("hivetab"), catTblDs("foo5")), acc.getLineages().get(1)); @@ -437,7 +369,7 @@ public void testHiveInHiveOut() throws Exception { Thread.sleep(5000); check(dsl(catTblDs("hivetab"), catTblDs("foo5")), acc.getLineages().get(3)); if (VERIFY_EXPECTED) { - verify(4 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); + verify(4 * N); } } @@ -449,74 +381,48 @@ public void testHdfsJdbcInJdbcOutTwoLevel() throws Exception { c.close(); Dataset df1 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in1.csv") - .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b2"); + .option("header", "true") + .csv(DATA_DIR + "/in1.csv") + .withColumnRenamed("c1", "a") + .withColumnRenamed("c2", "b2"); - Dataset df2 = spark.read() - .jdbc(db.getJdbcUrl(), "foo6", jdbcConnnProperties); + Dataset df2 = spark.read().jdbc(db.getJdbcUrl(), "foo6", jdbcConnnProperties); Dataset df3 = spark.read() - .option("header", "true").csv(DATA_DIR + "/in2.csv") - .withColumnRenamed("c1", "a").withColumnRenamed("c2", "b3"); + .option("header", "true") + .csv(DATA_DIR + "/in2.csv") + .withColumnRenamed("c1", "a") + .withColumnRenamed("c2", "b3"); Dataset df = df1.join(df2, "a").drop("id").join(df3, "a"); // SaveIntoDataSourceCommand // JDBCRelation input - df.write().mode(SaveMode.Overwrite).jdbc( - db.getJdbcUrl(), - "foo7", jdbcConnnProperties); + df.write().mode(SaveMode.Overwrite).jdbc(db.getJdbcUrl(), "foo7", jdbcConnnProperties); Thread.sleep(5000); check(dsl(pgDs("foo7"), hdfsDs("in1.csv"), hdfsDs("in2.csv"), pgDs("foo6")), acc.getLineages().get(0)); if (VERIFY_EXPECTED) { - verify(1 * N, postRequestedFor(urlEqualTo("/aspects?action=ingestProposal"))); - } - } - - private static void check(List expected, List actual) { - assertEquals(expected.size(), actual.size()); - for (int i = 0; i < expected.size(); i++) { - check(expected.get(i), actual.get(i)); + verify(1 * N); } } - private static void check(DatasetLineage expected, DatasetLineage actual) { - assertEquals(expected.getSink().toString(), actual.getSink().toString()); - assertEquals(dsToStrings(expected.getSources()), dsToStrings(actual.getSources())); - assertTrue(actual.getCallSiteShort().contains("TestSparkJobsLineage")); - } - - private static Set dsToStrings(Set datasets) { - return datasets.stream().map(x -> x.toString()).collect(Collectors.toSet()); - } - - private static DatasetLineage dsl(SparkDataset sink, SparkDataset... source) { - return dsl(null, sink, source); - } - - private static DatasetLineage dsl(String callSite, SparkDataset sink, SparkDataset... source) { - DatasetLineage lineage = new DatasetLineage(callSite, "unknownPlan", sink); - Arrays.asList(source).forEach(x -> lineage.addSource(x)); - return lineage; - } - - private static HdfsPathDataset hdfsDs(String fileName) { - return new HdfsPathDataset("file:" + abs(DATA_DIR + "/" + fileName)); - } + private static class DatasetLineageAccumulator implements LineageConsumer { - private static JdbcDataset pgDs(String tbl) { - return new JdbcDataset(db.getJdbcUrl(), tbl); - } + private final List lineages = new ArrayList<>(); - private static CatalogTableDataset catTblDs(String tbl) { - return new CatalogTableDataset(tbl(tbl)); - } + public void flushJobs() { + lineages.clear(); + } - private static String tbl(String tbl) { - return TEST_DB + "." + tbl; - } + public List getLineages() { + return Collections.unmodifiableList(lineages); + } - private static String abs(String relPath) { - return new File(relPath).getAbsolutePath(); + @Override + public void accept(LineageEvent e) { + if (e instanceof SQLQueryExecStartEvent) { + lineages.add(((SQLQueryExecStartEvent) e).getDatasetLineage()); + } + } } } \ No newline at end of file From aa5b3565d8cbee7d0cb7475be545f67248615368 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Wed, 29 Dec 2021 23:49:55 +0530 Subject: [PATCH 18/24] fix checkstyle and timing tests --- .../client/MetadataResponseFuture.java | 69 +++--- .../main/java/datahub/client/RestEmitter.java | 87 ++++---- .../datahub/client/RestEmitterConfig.java | 7 +- .../java/datahub/event/EventFormatter.java | 12 +- .../java/datahub/client/RestEmitterTest.java | 197 ++++++++++-------- .../datahub/event/EventFormatterTest.java | 19 +- .../lineage/consumer/impl/McpEmitter.java | 24 +++ 7 files changed, 230 insertions(+), 185 deletions(-) diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java index 8fc9acaac9697d..51126a1cdcbea1 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataResponseFuture.java @@ -16,18 +16,14 @@ public class MetadataResponseFuture implements Future { private final CountDownLatch responseLatch; private final ResponseMapper mapper; - @FunctionalInterface - public interface ResponseMapper { - MetadataWriteResponse map(HttpResponse httpResponse); + public MetadataResponseFuture(Future underlyingFuture, + AtomicReference responseAtomicReference, CountDownLatch responseLatch) { + this.requestFuture = underlyingFuture; + this.responseReference = responseAtomicReference; + this.responseLatch = responseLatch; + this.mapper = null; } - public MetadataResponseFuture(Future underlyingFuture, AtomicReference responseAtomicReference, CountDownLatch responseLatch) { - this.requestFuture = underlyingFuture; - this.responseReference = responseAtomicReference; - this.responseLatch = responseLatch; - this.mapper = null; - } - public MetadataResponseFuture(Future underlyingFuture, ResponseMapper mapper) { this.requestFuture = underlyingFuture; this.responseReference = null; @@ -35,38 +31,37 @@ public MetadataResponseFuture(Future underlyingFuture, ResponseMap this.mapper = mapper; } + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return requestFuture.cancel(mayInterruptIfRunning); + } @Override - public boolean cancel(boolean mayInterruptIfRunning) { - return requestFuture.cancel(mayInterruptIfRunning); - } + public boolean isCancelled() { + return requestFuture.isCancelled(); + } - @Override - public boolean isCancelled() { - return requestFuture.isCancelled(); - } + @Override + public boolean isDone() { + return requestFuture.isDone(); + } - @Override - public boolean isDone() { - return requestFuture.isDone(); + @SneakyThrows + @Override + public MetadataWriteResponse get() throws InterruptedException, ExecutionException { + HttpResponse response = requestFuture.get(); + if (mapper != null) { + return mapper.map(response); + } else { + // We wait for the callback to fill this out + responseLatch.await(); + return responseReference.get(); } - - @SneakyThrows - @Override - public MetadataWriteResponse get() throws InterruptedException, ExecutionException { - HttpResponse response = requestFuture.get(); - if (mapper != null) { - return mapper.map(response); - } else { - // We wait for the callback to fill this out - responseLatch.await(); - return responseReference.get(); - } } - @Override - public MetadataWriteResponse get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { + @Override + public MetadataWriteResponse get(long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { HttpResponse response = requestFuture.get(timeout, unit); if (mapper != null) { return mapper.map(response); @@ -77,4 +72,8 @@ public MetadataWriteResponse get(long timeout, TimeUnit unit) } } + @FunctionalInterface + public interface ResponseMapper { + MetadataWriteResponse map(HttpResponse httpResponse); + } } diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java index 7442e04ebde897..f1311c6a9cff8f 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java @@ -38,14 +38,12 @@ public class RestEmitter implements Emitter { private final CloseableHttpAsyncClient httpClient; private final EventFormatter eventFormatter; - public RestEmitter( - RestEmitterConfig config - ) { + public RestEmitter(RestEmitterConfig config) { this.config = config; // Override httpClient settings with RestEmitter configs if present if (config.getTimeoutSec() != null) { HttpAsyncClientBuilder httpClientBuilder = this.config.getAsyncHttpClientBuilder(); - httpClientBuilder.setDefaultRequestConfig(RequestConfig.custom() + httpClientBuilder.setDefaultRequestConfig(RequestConfig.custom() .setConnectTimeout(config.getTimeoutSec() * 1000) .setSocketTimeout(config.getTimeoutSec() * 1000) .build()); @@ -57,13 +55,52 @@ public RestEmitter( this.eventFormatter = this.config.getEventFormatter(); } + private static MetadataWriteResponse mapResponse(HttpResponse response) { + MetadataWriteResponse.MetadataWriteResponseBuilder builder = + MetadataWriteResponse.builder().underlyingResponse(response); + if ((response != null) && (response.getStatusLine() != null) && (response.getStatusLine().getStatusCode() == 200 + || response.getStatusLine().getStatusCode() == 201)) { + builder.success(true); + } else { + builder.success(false); + try { + ByteArrayOutputStream result = new ByteArrayOutputStream(); + InputStream contentStream = response.getEntity().getContent(); + byte[] buffer = new byte[1024]; + int length = contentStream.read(buffer); + while (length > 0) { + result.write(buffer, 0, length); + length = contentStream.read(buffer); + } + builder.serverException(result.toString("UTF-8")); + } catch (Exception e) { + // Catch all exceptions and still return a valid response object + log.warn("Wasn't able to convert response into a string", e); + } + } + return builder.build(); + } + + public static RestEmitter create(Consumer builderSupplier) { + RestEmitter restEmitter = new RestEmitter(RestEmitterConfig.builder().with(builderSupplier).build()); + return restEmitter; + } + + public static RestEmitter createWithDefaults() { + // No-op creator -> creates RestEmitter using default settings + return create(b -> { + }); + } + @Override - public Future emit(MetadataChangeProposalWrapper mcpw, Callback callback) throws IOException { + public Future emit(MetadataChangeProposalWrapper mcpw, + Callback callback) throws IOException { return emit(this.eventFormatter.convert(mcpw), callback); } @Override - public Future emit(MetadataChangeProposal mcp, Callback callback) throws IOException { + public Future emit(MetadataChangeProposal mcp, Callback callback) + throws IOException { DataMap map = new DataMap(); map.put("proposal", mcp.data()); String serializedMCP = dataTemplateCodec.mapToString(map); @@ -71,7 +108,8 @@ public Future emit(MetadataChangeProposal mcp, Callback postGeneric(String urlStr, String payloadJson, Object originalRequest, Callback callback) throws IOException { + private Future postGeneric(String urlStr, String payloadJson, Object originalRequest, + Callback callback) throws IOException { HttpPost httpPost = new HttpPost(urlStr); httpPost.setHeader("Content-Type", "application/json"); httpPost.setHeader("X-RestLi-Protocol-Version", "2.0.0"); @@ -113,31 +151,6 @@ public void cancelled() { return new MetadataResponseFuture(requestFuture, responseAtomicReference, responseLatch); } - private static MetadataWriteResponse mapResponse(HttpResponse response) { - MetadataWriteResponse.MetadataWriteResponseBuilder builder = MetadataWriteResponse.builder() - .underlyingResponse(response); - if ((response!= null) && (response.getStatusLine()!=null) && (response.getStatusLine().getStatusCode() == 200 || response.getStatusLine().getStatusCode() == 201)) { - builder.success(true); - } else { - builder.success(false); - try { - ByteArrayOutputStream result = new ByteArrayOutputStream(); - InputStream contentStream = response.getEntity().getContent(); - byte[] buffer = new byte[1024]; - for (int length; (length = contentStream.read(buffer)) != -1; ) { - result.write(buffer, 0, length); - } - builder.serverException(result.toString("UTF-8")); - } - catch (Exception e) { - // Catch all exceptions and still return a valid response object - log.warn("Wasn't able to convert response into a string", e); - } - } - return builder.build(); - } - - private Future getGeneric(String urlStr) throws IOException { HttpGet httpGet = new HttpGet(urlStr); httpGet.setHeader("Content-Type", "application/json"); @@ -152,16 +165,6 @@ public boolean testConnection() throws IOException, ExecutionException, Interrup return this.getGeneric(this.configUrl).get().isSuccess(); } - public static RestEmitter create(Consumer builderSupplier) { - RestEmitter restEmitter = new RestEmitter(RestEmitterConfig.builder().with(builderSupplier).build()); - return restEmitter; - } - - public static RestEmitter createWithDefaults() { - // No-op creator -> creates RestEmitter using default settings - return create(b -> {}); - } - @Override public void close() throws IOException { this.httpClient.close(); diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java index 9e3d3497c2e56c..a5cf03956e3324 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java @@ -15,10 +15,9 @@ @Builder public class RestEmitterConfig { - private static final int DEFAULT_CONNECT_TIMEOUT_SEC = 30; - private static final int DEFAULT_READ_TIMEOUT_SEC = 30; - private static final String DEFAULT_AUTH_TOKEN = null; - //private static final HttpClientFactory DEFAULT_HTTP_CLIENT_FACTORY = new HttpClientFactory() {}; + public static final int DEFAULT_CONNECT_TIMEOUT_SEC = 10; + public static final int DEFAULT_READ_TIMEOUT_SEC = 10; + public static final String DEFAULT_AUTH_TOKEN = null; @Builder.Default private final String gmsUrl = "http://localhost:8080"; diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java index d5fb14e3e0a606..3b08cea817d872 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/event/EventFormatter.java @@ -10,7 +10,6 @@ import java.io.IOException; import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; -import javax.print.URIException; import lombok.SneakyThrows; @@ -19,11 +18,8 @@ */ public class EventFormatter { - public enum Format { - PEGASUS_JSON, - }; - private final ObjectMapper objectMapper = new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL); + private final JacksonDataTemplateCodec dataTemplateCodec = new JacksonDataTemplateCodec(objectMapper.getFactory()); private final Format serializationFormat; @@ -49,9 +45,13 @@ public MetadataChangeProposal convert(MetadataChangeProposalWrapper mcpw) throws .setValue(ByteString.unsafeWrap(serializedAspect.getBytes(StandardCharsets.UTF_8)))); } break; - default: throw new EventValidationException("Cannot handle serialization format " + this.serializationFormat); + default: + throw new EventValidationException("Cannot handle serialization format " + this.serializationFormat); } return mcp; } + public enum Format { + PEGASUS_JSON, + } } diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java index 8aa15d09093cf6..e8476ccabcb262 100644 --- a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java @@ -7,8 +7,8 @@ import datahub.server.TestDataHubServer; import java.io.IOException; import java.io.InputStream; +import java.net.SocketTimeoutException; import java.net.URISyntaxException; -import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; @@ -21,6 +21,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import org.apache.http.HttpResponse; @@ -79,33 +81,27 @@ public void testPost() throws URISyntaxException, IOException { InputStream is = testPost.getEntity().getContent(); byte[] contentBytes = new byte[(int) testPost.getEntity().getContentLength()]; is.read(contentBytes); - String contentString = new String(contentBytes, Charset.forName("UTF-8")); - String expectedContent = - "{\"proposal\":{\"aspectName\":\"datasetProperties\"," - + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)\"," - + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" - + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset\\\"}\"}}}"; + String contentString = new String(contentBytes, StandardCharsets.UTF_8); + String expectedContent = "{\"proposal\":{\"aspectName\":\"datasetProperties\"," + + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)\"," + + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset\\\"}\"}}}"; Assert.assertEquals(expectedContent, contentString); } @Test public void testExceptions() throws URISyntaxException, IOException, ExecutionException, InterruptedException { - RestEmitter emitter = RestEmitter.create($ -> $ - .asyncHttpClientBuilder(mockHttpClientFactory) - ); + RestEmitter emitter = RestEmitter.create($ -> $.asyncHttpClientBuilder(mockHttpClientFactory)); - MetadataChangeProposalWrapper mcp = MetadataChangeProposalWrapper.create(b -> b - .entityType("dataset") - .changeType(ChangeType.UPSERT) - .aspect(new DatasetProperties().setDescription("Test Dataset")) - .entityUrn("urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)")); + MetadataChangeProposalWrapper mcp = MetadataChangeProposalWrapper.create(b -> b.entityType("dataset") + .changeType(ChangeType.UPSERT) + .aspect(new DatasetProperties().setDescription("Test Dataset")) + .entityUrn("urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar,PROD)")); Future mockFuture = Mockito.mock(Future.class); - Mockito.when(mockClient.execute(Mockito.any(), Mockito.any())) - .thenReturn(mockFuture); - Mockito.when(mockFuture.get()) - .thenThrow(new ExecutionException("Test execution exception", null)); + Mockito.when(mockClient.execute(Mockito.any(), Mockito.any())).thenReturn(mockFuture); + Mockito.when(mockFuture.get()).thenThrow(new ExecutionException("Test execution exception", null)); try { emitter.emit(mcp, null).get(); Assert.fail("should not be here"); @@ -116,17 +112,12 @@ public void testExceptions() throws URISyntaxException, IOException, ExecutionEx @Test public void testExtraHeaders() throws Exception { - RestEmitter emitter = RestEmitter.create(b -> b - .asyncHttpClientBuilder(mockHttpClientFactory) + RestEmitter emitter = RestEmitter.create(b -> b.asyncHttpClientBuilder(mockHttpClientFactory) .extraHeaders(Collections.singletonMap("Test-Header", "Test-Value"))); MetadataChangeProposalWrapper mcpw = MetadataChangeProposalWrapper.create( - b -> b.entityType("dataset") - .entityUrn("urn:li:dataset:foo") - .aspect(new DatasetProperties()) - ); + b -> b.entityType("dataset").entityUrn("urn:li:dataset:foo").aspect(new DatasetProperties())); Future mockFuture = Mockito.mock(Future.class); - Mockito.when(mockClient.execute(Mockito.any(), Mockito.any())) - .thenReturn(mockFuture); + Mockito.when(mockClient.execute(Mockito.any(), Mockito.any())).thenReturn(mockFuture); emitter.emit(mcpw, null); Mockito.verify(mockClient).execute(postArgumentCaptor.capture(), callbackCaptor.capture()); FutureCallback callback = callbackCaptor.getValue(); @@ -150,27 +141,22 @@ public void mockServerTest() throws InterruptedException, ExecutionException, IO public void multithreadedTestExecutors() throws Exception { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b - .gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); testDataHubServer.getMockServer() - .when( - request() - .withMethod("POST") + .when(request().withMethod("POST") .withPath("/aspects") .withQueryStringParameter("action", "ingestProposal") - .withHeader("Content-type", "application/json"), - Times.unlimited()) - .respond(org.mockserver.model.HttpResponse.response() - .withStatusCode(200)); + .withHeader("Content-type", "application/json"), Times.unlimited()) + .respond(org.mockserver.model.HttpResponse.response().withStatusCode(200)); ExecutorService executor = Executors.newFixedThreadPool(10); ArrayList results = new ArrayList(); Random random = new Random(); int testIteration = random.nextInt(); int numRequests = 100; - for (int i=0; i { + results.add(executor.submit(() -> { try { Thread.sleep(random.nextInt(100)); MetadataChangeProposalWrapper mcp = @@ -191,33 +177,32 @@ public void multithreadedTestExecutors() throws Exception { Assert.fail(e.getMessage()); } }); - RequestDefinition[] recordedRequests = testDataHubServer.getMockServer() - .retrieveRecordedRequests( - request() - .withPath("/aspects") - .withMethod("POST") - ); + RequestDefinition[] recordedRequests = + testDataHubServer.getMockServer().retrieveRecordedRequests(request().withPath("/aspects").withMethod("POST")); Assert.assertEquals(100, recordedRequests.length); - List requests = Arrays.stream(recordedRequests).sequential() + List requests = Arrays.stream(recordedRequests) + .sequential() .filter(x -> x instanceof HttpRequest) .map(x -> (HttpRequest) x) .collect(Collectors.toList()); ObjectMapper mapper = new ObjectMapper(); for (int i = 0; i < numRequests; ++i) { - String expectedContent = String.format("{\"proposal\":{\"aspectName\":\"datasetProperties\"," + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar-%d,PROD)\"," - + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset %d\\\"}\"}}}", i, testIteration); + String expectedContent = String.format("{\"proposal\":{\"aspectName\":\"datasetProperties\"," + + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar-%d,PROD)\"," + + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset %d\\\"}\"}}}", i, testIteration); Assert.assertEquals(requests.stream().filter(x -> { String bodyString = ""; try { - bodyString = mapper.writeValueAsString(mapper.readValue(x.getBodyAsString().getBytes(StandardCharsets.UTF_8), Map.class)); + bodyString = mapper.writeValueAsString( + mapper.readValue(x.getBodyAsString().getBytes(StandardCharsets.UTF_8), Map.class)); } catch (IOException ioException) { return false; } return bodyString.equals(expectedContent); }).count(), 1); } - } private MetadataChangeProposalWrapper getMetadataChangeProposalWrapper(String description, String entityUrn) { @@ -233,29 +218,24 @@ private MetadataChangeProposalWrapper getMetadataChangeProposalWrapper(String de public void multithreadedTestSingleThreadCaller() throws Exception { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b - .gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); testDataHubServer.getMockServer() - .when( - request() - .withMethod("POST") - .withPath("/aspects") - .withQueryStringParameter("action", "ingestProposal") - .withHeader("Content-type", "application/json"), - Times.unlimited()) - .respond(org.mockserver.model.HttpResponse.response() - .withStatusCode(200)); + .when(request().withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withHeader("Content-type", "application/json"), Times.unlimited()) + .respond(org.mockserver.model.HttpResponse.response().withStatusCode(200)); ArrayList results = new ArrayList(); Random random = new Random(); int testIteration = random.nextInt(); int numRequests = 100; - for (int i=0; i future = emitter.emit(mcp, null); - results.add(future); + results.add(future); } results.forEach(x -> { try { @@ -264,54 +244,46 @@ public void multithreadedTestSingleThreadCaller() throws Exception { Assert.fail(e.getMessage()); } }); - RequestDefinition[] recordedRequests = testDataHubServer.getMockServer() - .retrieveRecordedRequests( - request() - .withPath("/aspects") - .withMethod("POST") - ); + RequestDefinition[] recordedRequests = + testDataHubServer.getMockServer().retrieveRecordedRequests(request().withPath("/aspects").withMethod("POST")); Assert.assertEquals(numRequests, recordedRequests.length); - List requests = Arrays.stream(recordedRequests).sequential() + List requests = Arrays.stream(recordedRequests) + .sequential() .filter(x -> x instanceof HttpRequest) .map(x -> (HttpRequest) x) .collect(Collectors.toList()); ObjectMapper mapper = new ObjectMapper(); for (int i = 0; i < numRequests; ++i) { - String expectedContent = String.format("{\"proposal\":{\"aspectName\":\"datasetProperties\"," + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar-%d,PROD)\"," - + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset %d\\\"}\"}}}", i, testIteration); + String expectedContent = String.format("{\"proposal\":{\"aspectName\":\"datasetProperties\"," + + "\"entityUrn\":\"urn:li:dataset:(urn:li:dataPlatform:hive,foo.bar-%d,PROD)\"," + + "\"entityType\":\"dataset\",\"changeType\":\"UPSERT\",\"aspect\":{\"contentType\":\"application/json\"" + + ",\"value\":\"{\\\"description\\\":\\\"Test Dataset %d\\\"}\"}}}", i, testIteration); Assert.assertEquals(requests.stream().filter(x -> { String bodyString = ""; try { - bodyString = mapper.writeValueAsString(mapper.readValue(x.getBodyAsString().getBytes(StandardCharsets.UTF_8), Map.class)); + bodyString = mapper.writeValueAsString( + mapper.readValue(x.getBodyAsString().getBytes(StandardCharsets.UTF_8), Map.class)); } catch (IOException ioException) { return false; } return bodyString.equals(expectedContent); }).count(), 1); } - } - @Test public void testCallback() throws Exception { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b - .gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); testDataHubServer.getMockServer() - .when( - request() - .withMethod("POST") - .withPath("/aspects") - .withQueryStringParameter("action", "ingestProposal") - .withHeader("Content-type", "application/json"), - Times.unlimited()) - .respond(org.mockserver.model.HttpResponse.response() - .withStatusCode(500) - .withBody("exception")); + .when(request().withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withHeader("Content-type", "application/json"), Times.unlimited()) + .respond(org.mockserver.model.HttpResponse.response().withStatusCode(500).withBody("exception")); MetadataChangeProposalWrapper mcpw = getMetadataChangeProposalWrapper("Test Dataset", "urn:li:dataset:foo"); AtomicReference callbackResponse = new AtomicReference<>(); @@ -333,8 +305,61 @@ public void onFailure(Throwable exception) { latch.await(); Assert.assertEquals(callbackResponse.get(), future.get()); + } + + @Test + public void testTimeoutOnGet() { + TestDataHubServer testDataHubServer = new TestDataHubServer(); + Integer port = testDataHubServer.getMockServer().getPort(); + RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + testDataHubServer.getMockServer().reset(); + testDataHubServer.getMockServer() + .when(request().withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withHeader("Content-type", "application/json"), Times.once()) + .respond(org.mockserver.model.HttpResponse.response() + .withStatusCode(200) + .withDelay(TimeUnit.SECONDS, RestEmitterConfig.DEFAULT_READ_TIMEOUT_SEC + 3)); + MetadataChangeProposalWrapper mcpw = getMetadataChangeProposalWrapper("Test Dataset", "urn:li:dataset:foo"); + try { + long startTime = System.currentTimeMillis(); + MetadataWriteResponse response = emitter.emit(mcpw, null).get(); + long duration = (long) ((System.currentTimeMillis() - startTime) / 1000.0); + Assert.fail("Should not succeed with duration " + duration); + } catch (Exception ioe) { + Assert.assertTrue(ioe instanceof ExecutionException); + Assert.assertTrue(((ExecutionException) ioe).getCause() instanceof SocketTimeoutException); + } } + @Test + public void testTimeoutOnGetWithTimeout() { + TestDataHubServer testDataHubServer = new TestDataHubServer(); + Integer port = testDataHubServer.getMockServer().getPort(); + RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + + testDataHubServer.getMockServer().reset(); + testDataHubServer.getMockServer() + .when(request().withMethod("POST") + .withPath("/aspects") + .withQueryStringParameter("action", "ingestProposal") + .withHeader("Content-type", "application/json"), Times.once()) + .respond(org.mockserver.model.HttpResponse.response() + .withStatusCode(200) + .withDelay(TimeUnit.SECONDS, RestEmitterConfig.DEFAULT_READ_TIMEOUT_SEC + 3)); + + MetadataChangeProposalWrapper mcpw = getMetadataChangeProposalWrapper("Test Dataset", "urn:li:dataset:foo"); + try { + long startTime = System.currentTimeMillis(); + MetadataWriteResponse response = + emitter.emit(mcpw, null).get(RestEmitterConfig.DEFAULT_READ_TIMEOUT_SEC - 3, TimeUnit.SECONDS); + long duration = (long) ((System.currentTimeMillis() - startTime) / 1000.0); + Assert.fail("Should not succeed with duration " + duration); + } catch (Exception ioe) { + Assert.assertTrue(ioe instanceof TimeoutException); + } + } } \ No newline at end of file diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java index aee662251ad16c..0e70b99f362790 100644 --- a/metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/event/EventFormatterTest.java @@ -1,12 +1,10 @@ package datahub.event; -import com.linkedin.common.urn.Urn; import com.linkedin.dataset.DatasetProperties; -import com.linkedin.events.metadata.ChangeType; import com.linkedin.mxe.MetadataChangeProposal; import java.io.IOException; import java.net.URISyntaxException; -import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import org.junit.Test; import org.testng.Assert; @@ -15,19 +13,16 @@ public class EventFormatterTest { @Test public void testPartialMCPW() throws URISyntaxException, IOException, EventValidationException { - MetadataChangeProposalWrapper metadataChangeProposalWrapper = - MetadataChangeProposalWrapper.builder() - .entityType("dataset") - .entityUrn("urn:li:foo") - .aspect(new DatasetProperties() - .setDescription("A test dataset")) - .build(); + MetadataChangeProposalWrapper metadataChangeProposalWrapper = MetadataChangeProposalWrapper.builder() + .entityType("dataset") + .entityUrn("urn:li:foo") + .aspect(new DatasetProperties().setDescription("A test dataset")) + .build(); MetadataChangeProposalWrapper.validate(metadataChangeProposalWrapper); EventFormatter eventFormatter = new EventFormatter(); MetadataChangeProposal mcp = eventFormatter.convert(metadataChangeProposalWrapper); Assert.assertEquals(mcp.getAspect().getContentType(), "application/json"); - String content = mcp.getAspect().getValue().asString(Charset.forName("UTF-8")); + String content = mcp.getAspect().getValue().asString(StandardCharsets.UTF_8); Assert.assertEquals(content, "{\"description\":\"A test dataset\"}"); } - } diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java index 9cfeedc9c9a83f..4e1117b13d127e 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java @@ -1,13 +1,19 @@ package com.linkedin.datahub.lineage.consumer.impl; import datahub.client.Emitter; +import datahub.client.MetadataWriteResponse; import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; import java.util.List; +import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import org.apache.spark.SparkConf; import org.apache.spark.SparkEnv; @@ -29,6 +35,23 @@ public class McpEmitter implements LineageConsumer { private void emit(List mcps) { Emitter emitter = emitter(); if (emitter != null) { + mcps.stream().map(mcp -> { + try { + return emitter.emit(mcp, null); + } catch (IOException ioException) { + log.error("Failed to emit metadata to DataHub", ioException); + return null; + } + }).filter(Objects::nonNull).collect(Collectors.toList()) + .forEach(future -> { + try { + future.get(); + } catch (InterruptedException | ExecutionException e) { + // log error, but don't impact thread + log.error("Failed to emit metadata to DataHub", e); + } + }); + /** mcps.forEach(mcp -> { log.debug("Emitting {}", mcp); try { @@ -42,6 +65,7 @@ private void emit(List mcps) { p.close(); } }); + **/ } } From fb35e0ec403e770722533d511fb17e1f81eb9ad2 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Wed, 29 Dec 2021 23:59:28 +0530 Subject: [PATCH 19/24] fix checkstyle --- .../lineage/consumer/impl/McpEmitter.java | 39 ++++--------------- 1 file changed, 7 insertions(+), 32 deletions(-) diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java index 4e1117b13d127e..1394dc99c7215d 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java @@ -1,28 +1,20 @@ package com.linkedin.datahub.lineage.consumer.impl; +import com.linkedin.datahub.lineage.spark.model.LineageConsumer; +import com.linkedin.datahub.lineage.spark.model.LineageEvent; +import com.linkedin.mxe.MetadataChangeProposal; import datahub.client.Emitter; -import datahub.client.MetadataWriteResponse; +import datahub.client.RestEmitter; import java.io.IOException; -import java.io.PrintWriter; -import java.io.StringWriter; import java.util.List; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; - import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import lombok.extern.slf4j.Slf4j; import org.apache.spark.SparkConf; import org.apache.spark.SparkEnv; -import com.linkedin.datahub.lineage.spark.model.LineageConsumer; -import com.linkedin.datahub.lineage.spark.model.LineageEvent; -import com.linkedin.mxe.MetadataChangeProposal; - -import datahub.client.RestEmitter; -import lombok.extern.slf4j.Slf4j; @Slf4j public class McpEmitter implements LineageConsumer { @@ -42,8 +34,7 @@ private void emit(List mcps) { log.error("Failed to emit metadata to DataHub", ioException); return null; } - }).filter(Objects::nonNull).collect(Collectors.toList()) - .forEach(future -> { + }).filter(Objects::nonNull).collect(Collectors.toList()).forEach(future -> { try { future.get(); } catch (InterruptedException | ExecutionException e) { @@ -51,21 +42,6 @@ private void emit(List mcps) { log.error("Failed to emit metadata to DataHub", e); } }); - /** - mcps.forEach(mcp -> { - log.debug("Emitting {}", mcp); - try { - emitter.emit(mcp, null).get(); - } catch (IOException | InterruptedException | ExecutionException e) { - // log error, but don't impact thread - StringWriter s = new StringWriter(); - PrintWriter p = new PrintWriter(s); - e.printStackTrace(p); - log.error(s.toString()); - p.close(); - } - }); - **/ } } @@ -77,8 +53,7 @@ private Emitter emitter() { if (conf.contains(GMS_URL_KEY)) { String gmsUrl = conf.get(GMS_URL_KEY); log.debug("REST emitter configured with GMS url " + gmsUrl); - return RestEmitter.create($ -> $ - .gmsUrl(gmsUrl)); + return RestEmitter.create($ -> $.gmsUrl(gmsUrl)); } log.error("GMS URL not configured."); From a2cbb9f77858ff4ee6e462656949dfe609e22b70 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Thu, 30 Dec 2021 22:51:49 +0530 Subject: [PATCH 20/24] more simplifications --- .../main/java/datahub/client/Callback.java | 4 ++-- .../src/main/java/datahub/client/Emitter.java | 4 ++-- .../datahub/client/MetadataWriteResponse.java | 6 ----- .../main/java/datahub/client/RestEmitter.java | 24 ++++++++++++++----- .../java/datahub/client/RestEmitterTest.java | 2 +- 5 files changed, 23 insertions(+), 17 deletions(-) diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java index cee3f95f063857..84fe9cef0817c0 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Callback.java @@ -3,7 +3,7 @@ import javax.annotation.Nullable; -public interface Callback { +public interface Callback { /** * Called when the client request has completed. @@ -11,7 +11,7 @@ public interface Callback { * this was a successfully processed request or not. * @param response */ - void onCompletion(@Nullable T response); + void onCompletion(@Nullable MetadataWriteResponse response); /** * Called when the client request has thrown an exception before completion. diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java index 3d3d359f7f1db3..24eec2df1a6ae9 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java @@ -10,9 +10,9 @@ public interface Emitter extends Closeable { - Future emit(MetadataChangeProposalWrapper mcpw, Callback callback) throws IOException; + Future emit(MetadataChangeProposalWrapper mcpw, Callback callback) throws IOException; - Future emit(MetadataChangeProposal mcp, Callback callback) throws IOException; + Future emit(MetadataChangeProposal mcp, Callback callback) throws IOException; boolean testConnection() throws IOException, ExecutionException, InterruptedException; diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java index d0abf747a8e338..16aee6c0ff2748 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java @@ -1,6 +1,5 @@ package datahub.client; -import java.util.Map; import lombok.Builder; import lombok.Value; @@ -15,8 +14,6 @@ public class MetadataWriteResponse { @Builder.Default boolean success = true; - Map responseMetadata; - /** * If the write failed due to an exception thrown by the server * and we have access to it, then we store the stack trace here @@ -29,7 +26,4 @@ public class MetadataWriteResponse { */ Object underlyingResponse; - - Throwable clientException; - } diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java index f1311c6a9cff8f..70d54186bf7dac 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java @@ -94,12 +94,12 @@ public static RestEmitter createWithDefaults() { @Override public Future emit(MetadataChangeProposalWrapper mcpw, - Callback callback) throws IOException { + Callback callback) throws IOException { return emit(this.eventFormatter.convert(mcpw), callback); } @Override - public Future emit(MetadataChangeProposal mcp, Callback callback) + public Future emit(MetadataChangeProposal mcp, Callback callback) throws IOException { DataMap map = new DataMap(); map.put("proposal", mcp.data()); @@ -109,7 +109,7 @@ public Future emit(MetadataChangeProposal mcp, Callback postGeneric(String urlStr, String payloadJson, Object originalRequest, - Callback callback) throws IOException { + Callback callback) throws IOException { HttpPost httpPost = new HttpPost(urlStr); httpPost.setHeader("Content-Type", "application/json"); httpPost.setHeader("X-RestLi-Protocol-Version", "2.0.0"); @@ -133,18 +133,30 @@ public void completed(HttpResponse response) { } responseLatch.countDown(); if (callback != null) { - callback.onCompletion(writeResponse); + try { + callback.onCompletion(writeResponse); + } catch (Exception e) { + log.error("Error executing user callback on completion.", e); + } } } @Override public void failed(Exception ex) { - callback.onFailure(ex); + try { + callback.onFailure(ex); + } catch (Exception e) { + log.error("Error executing user callback on failure.", e); + } } @Override public void cancelled() { - callback.onFailure(new RuntimeException("Cancelled")); + try { + callback.onFailure(new RuntimeException("Cancelled")); + } catch (Exception e) { + log.error("Error executing user callback on failure due to cancellation.", e); + } } }; Future requestFuture = httpClient.execute(httpPost, httpCallback); diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java index e8476ccabcb262..45d3ecf451ccbe 100644 --- a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java @@ -288,7 +288,7 @@ public void testCallback() throws Exception { MetadataChangeProposalWrapper mcpw = getMetadataChangeProposalWrapper("Test Dataset", "urn:li:dataset:foo"); AtomicReference callbackResponse = new AtomicReference<>(); CountDownLatch latch = new CountDownLatch(1); - Future future = emitter.emit(mcpw, new Callback() { + Future future = emitter.emit(mcpw, new Callback() { @Override public void onCompletion(MetadataWriteResponse response) { callbackResponse.set(response); From 0d53dd17aa084f53ef13bac8028f49142d879886 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Fri, 31 Dec 2021 19:47:26 +0530 Subject: [PATCH 21/24] refactors --- docs-website/sidebars.js | 6 + metadata-ingestion/README.md | 5 +- metadata-ingestion/as-a-library.md | 128 ++++++++++++++++++ metadata-ingestion/transformers.md | 2 +- metadata-integration/java/as-a-library.md | 112 +++++++++++++++ .../client/{ => rest}/RestEmitter.java | 10 +- .../client/{ => rest}/RestEmitterConfig.java | 4 +- .../client/{ => rest}/RestEmitterTest.java | 18 ++- .../lineage/consumer/impl/McpEmitter.java | 4 +- 9 files changed, 270 insertions(+), 19 deletions(-) create mode 100644 metadata-ingestion/as-a-library.md create mode 100644 metadata-integration/java/as-a-library.md rename metadata-integration/java/datahub-client/src/main/java/datahub/client/{ => rest}/RestEmitter.java (95%) rename metadata-integration/java/datahub-client/src/main/java/datahub/client/{ => rest}/RestEmitterConfig.java (95%) rename metadata-integration/java/datahub-client/src/test/java/datahub/client/{ => rest}/RestEmitterTest.java (96%) diff --git a/docs-website/sidebars.js b/docs-website/sidebars.js index 2f59da6a10f8ab..642cbfd99a8e6b 100644 --- a/docs-website/sidebars.js +++ b/docs-website/sidebars.js @@ -85,6 +85,12 @@ module.exports = { { Sinks: list_ids_in_directory("metadata-ingestion/sink_docs"), }, + { + "Custom Integrations": [ + "metadata-ingestion/as-a-library", + "metadata-integration/java/as-a-library", + ], + }, { Scheduling: [ "metadata-ingestion/schedule_docs/intro", diff --git a/metadata-ingestion/README.md b/metadata-ingestion/README.md index aad490159905a2..8911f85517f991 100644 --- a/metadata-ingestion/README.md +++ b/metadata-ingestion/README.md @@ -186,10 +186,7 @@ Check out the [transformers guide](./transformers.md) for more info! ## Using as a library -In some cases, you might want to construct the MetadataChangeEvents yourself but still use this framework to emit that metadata to DataHub. In this case, take a look at the emitter interfaces, which can easily be imported and called from your own code. - -- [DataHub emitter via REST](./src/datahub/emitter/rest_emitter.py) (same requirements as `datahub-rest`). -- [DataHub emitter via Kafka](./src/datahub/emitter/kafka_emitter.py) (same requirements as `datahub-kafka`). +In some cases, you might want to construct Metadata events directly and use programmatic ways to emit that metadata to DataHub. In this case, take a look at the [Python emitter](./as-a-library.md) and the [Java emitter](../metadata-integration/java/as-a-library.md) libraries which can be called from your own code. ### Programmatic Pipeline In some cases, you might want to configure and run a pipeline entirely from within your custom python script. Here is an example of how to do it. diff --git a/metadata-ingestion/as-a-library.md b/metadata-ingestion/as-a-library.md new file mode 100644 index 00000000000000..b469b2470bf56f --- /dev/null +++ b/metadata-ingestion/as-a-library.md @@ -0,0 +1,128 @@ +# Python Emitter + +In some cases, you might want to construct Metadata events directly and use programmatic ways to emit that metadata to DataHub. Use-cases are typically push-based and include emitting metadata events from CI/CD pipelines, custom orchestrators etc. + +The `acryl-datahub` Python package offers REST and Kafka emitter API-s, which can easily be imported and called from your own code. + +## Installation + +Follow the installation guide for the main `acryl-datahub` package [here](./README.md#install-from-pypi). Read on for emitter specific installation instructions. +## REST Emitter + +The REST emitter is a thin wrapper on top of the `requests` module and offers a blocking interface for sending metadata events over HTTP. Use this when simplicity and acknowledgement of metadata being persisted to DataHub's metadata store is more important than throughput of metadata emission. Also use this when read-after-write scenarios exist, e.g. writing metadata and then immediately reading it back. + +### Installation + +```console +pip install -U `acryl-datahub[datahub-rest]` +``` + +### Example Usage +```python +import datahub.emitter.mce_builder as builder +from datahub.emitter.mcp import MetadataChangeProposalWrapper +from datahub.metadata.schema_classes import ChangeTypeClass, DatasetPropertiesClass + +from datahub.emitter.rest_emitter import DatahubRestEmitter + +# Create an emitter to DataHub over REST +emitter = DatahubRestEmitter(gms_server="http://localhost:8080", extra_headers={}) + +# Test the connection +emitter.test_connection() + +# Construct a dataset properties object +dataset_properties = DatasetPropertiesClass(description="This table stored the canonical User profile", + customProperties={ + "governance": "ENABLED" + }) + +# Construct a MetadataChangeProposalWrapper object. +metadata_event = MetadataChangeProposalWrapper( + entityType="dataset", + changeType=ChangeTypeClass.UPSERT, + entityUrn=builder.make_dataset_urn("bigquery", "my-project.my-dataset.user-table"), + aspectName="datasetProperties", + aspect=dataset_properties, +) + +# Emit metadata! This is a blocking call +emitter.emit(metadata_event) +``` + +Other examples: +- [lineage_emitter_mcpw_rest.py](./examples/library/lineage_emitter_mcpw_rest.py) - emits simple bigquery table-to-table (dataset-to-dataset) lineage via REST as MetadataChangeProposalWrapper. + +### Emitter Code + +If you're interested in looking at the REST emitter code, it is available [here](./src/datahub/emitter/rest_emitter.py) + +## Kafka Emitter + +The Kafka emitter is a thin wrapper on top of the SerializingProducer class from `confluent-kafka` and offers a non-blocking interface for sending metadata events to DataHub. Use this when you want to decouple your metadata producer from the uptime of your datahub metadata server by utilizing Kafka as a highly available message bus. For example, if your DataHub metadata service is down due to planned or unplanned outages, you can still continue to collect metadata from your mission critical systems by sending it to Kafka. Also use this emitter when throughput of metadata emission is more important than acknowledgement of metadata being persisted to DataHub's backend store. + +**_Note_**: The Kafka emitter uses Avro to serialize the Metadata events to Kafka. Changing the serializer will result in unprocessable events as DataHub currently expects the metadata events over Kafka to be serialized in Avro. + +### Installation + +```console +# For emission over Kafka +pip install -U `acryl-datahub[datahub-kafka]` +``` + + +### Example Usage +```python +import datahub.emitter.mce_builder as builder +from datahub.emitter.mcp import MetadataChangeProposalWrapper +from datahub.metadata.schema_classes import ChangeTypeClass, DatasetPropertiesClass + +from datahub.emitter.kafka_emitter import DatahubKafkaEmitter, KafkaEmitterConfig +# Create an emitter to Kafka +kafka_config = { + "connection": { + "bootstrap": "localhost:9092", + "schema_registry_url": "http://localhost:8081", + "schema_registry_config": {}, # schema_registry configs passed to underlying schema registry client + "producer_config": {}, # extra producer configs passed to underlying kafka producer + } +} + +emitter = DatahubKafkaEmitter( + KafkaEmitterConfig.parse_obj(kafka_config) +) + +# Construct a dataset properties object +dataset_properties = DatasetPropertiesClass(description="This table stored the canonical User profile", + customProperties={ + "governance": "ENABLED" + }) + +# Construct a MetadataChangeProposalWrapper object. +metadata_event = MetadataChangeProposalWrapper( + entityType="dataset", + changeType=ChangeTypeClass.UPSERT, + entityUrn=builder.make_dataset_urn("bigquery", "my-project.my-dataset.user-table"), + aspectName="datasetProperties", + aspect=dataset_properties, +) + + +# Emit metadata! This is a non-blocking call +emitter.emit( + metadata_event, + callback=lambda exc, message: print(f"Message sent to topic:{message.topic()}, partition:{message.partition()}, offset:{message.offset()}") if message else print(f"Failed to send with: {exc}") +) + +#Send all pending events +emitter.flush() +``` + +### Emitter Code +If you're interested in looking at the Kafka emitter code, it is available [here](./src/datahub/emitter/kafka_emitter.py) + +## Other Languages + +Emitter API-s are also supported for: +- [Java](../metadata-integration/java/as-a-library.md) + diff --git a/metadata-ingestion/transformers.md b/metadata-ingestion/transformers.md index 5534ef15c12682..b15d3fe8ba821b 100644 --- a/metadata-ingestion/transformers.md +++ b/metadata-ingestion/transformers.md @@ -1,4 +1,4 @@ -# Using transformers +# Transformers ## What’s a transformer? diff --git a/metadata-integration/java/as-a-library.md b/metadata-integration/java/as-a-library.md new file mode 100644 index 00000000000000..65f5a4dacf4381 --- /dev/null +++ b/metadata-integration/java/as-a-library.md @@ -0,0 +1,112 @@ +# Java Emitter + +In some cases, you might want to construct Metadata events directly and use programmatic ways to emit that metadata to DataHub. Use-cases are typically push-based and include emitting metadata events from CI/CD pipelines, custom orchestrators etc. + +The [`io.acryl:datahub-client`](https://mvnrepository.com/artifact/io.acryl/datahub-client) Java package offers REST emitter API-s, which can be easily used to emit metadata from your JVM-based systems. For example, the Spark lineage integration uses the Java emitter to emit metadata events from Spark jobs. + + +## Installation + +Follow the specific instructions for your build system to declare a dependency on the appropriate version of the package. + +**_Note_**: Check the [Maven repository](https://mvnrepository.com/artifact/io.acryl/datahub-client) for the latest version of the package before following the instructions below. + +### Gradle +Add the following to your build.gradle. +```gradle +implementation 'io.acryl:datahub-client:0.0.1' +``` +### Maven +Add the following to your `pom.xml`. +```xml + + + io.acryl + datahub-client + + 0.0.1 + +``` + +## REST Emitter + +The REST emitter is a thin wrapper on top of the [`Apache HttpClient`](https://hc.apache.org/httpcomponents-client-4.5.x/index.html) library. It supports non-blocking emission of metadata and handles the details of JSON serialization of metadata aspects over the wire. + +Constructing a REST Emitter follows a lambda-based fluent builder pattern. The config parameters mirror the Python emitter [configuration](../../metadata-ingestion/sink_docs/datahub.md#config-details) for the most part. In addition, you can also customize the HttpClient that is constructed under the hood by passing in customizations to the HttpClient builder. +```java +import datahub.client.rest.RestEmitter; +//... +RestEmitter emitter = RestEmitter.create(b -> b + .server("http://localhost:8080") +//Auth token for Managed DataHub .token(AUTH_TOKEN_IF_NEEDED) +//Override default timeout of 10 seconds .timeoutSec(OVERRIDE_DEFAULT_TIMEOUT_IN_SECONDS) +//Add additional headers .extraHeaders(Collections.singletonMap("Session-token", "MY_SESSION")) +// Customize HttpClient's connection ttl .customizeHttpAsyncClient(c -> c.setConnectionTimeToLive(30, TimeUnit.SECONDS)) + ); +``` + +### Usage + +```java +import com.linkedin.dataset.DatasetProperties; +import com.linkedin.events.metadata.ChangeType; +import datahub.event.MetadataChangeProposalWrapper; +import datahub.client.rest.RestEmitter; +import datahub.client.Callback; +// ... followed by + +// Creates the emitter with the default coordinates and settings +RestEmitter emitter = RestEmitter.createWithDefaults(); + +MetadataChangeProposalWrapper mcpw = MetadataChangeProposalWrapper.builder() + .entityType("dataset") + .changeType(ChangeType.UPSERT) + .aspect(new DatasetProperties().setDescription("This is the canonical User profile dataset")) + .entityUrn("urn:li:dataset:(urn:li:dataPlatform:bigquery,my-project.my-dataset.user-table,PROD)") + .build(); + +// Blocking call using future +Future requestFuture = emitter.emit(mcpw, null).get(); + +// Non-blocking using callback +emitter.emit(mcpw, new Callback() { + @Override + public void onCompletion(MetadataWriteResponse response) { + if (response.isSuccess()) { + System.out.println(String.format("Successfully emitted metadata event for %s", mcpw.getEntityUrn())); + } else { + // Get the underlying http response + HttpResponse httpResponse = (HttpResponse) response.getUnderlyingResponse(); + System.out.println(String.format("Failed to emit metadata event for %s, aspect: %s with status code: %d", + mcpw.getEntityUrn(), mcpw.getAspectName(), httpResponse.getStatusLine().getStatusCode())); + // Print the server side exception if it was captured + if (response.getServerException() != null) { + System.out.println(String.format("Server side exception was %s", response.getServerException())); + } + } + } + + @Override + public void onFailure(Throwable exception) { + System.out.println( + String.format("Failed to emit metadata event for %s, aspect: %s due to %s", mcpw.getEntityUrn(), + mcpw.getAspectName(), exception.getMessage())); + } + }); +``` + +### Emitter Code + +If you're interested in looking at the REST emitter code, it is available [here](./datahub-client/src/main/java/datahub/client/rest/RestEmitter.java). + +## Kafka Emitter + +The Java package doesn't currently support a Kafka emitter, but this will be available shortly. + + +## Other Languages + +Emitter API-s are also supported for: +- [Python](../../metadata-ingestion/as-a-library.md) + + diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitter.java similarity index 95% rename from metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java rename to metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitter.java index 70d54186bf7dac..29ef68bbb692cc 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitter.java @@ -1,10 +1,14 @@ -package datahub.client; +package datahub.client.rest; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.data.DataMap; import com.linkedin.data.template.JacksonDataTemplateCodec; import com.linkedin.mxe.MetadataChangeProposal; +import datahub.client.Callback; +import datahub.client.Emitter; +import datahub.client.MetadataResponseFuture; +import datahub.client.MetadataWriteResponse; import datahub.event.EventFormatter; import datahub.event.MetadataChangeProposalWrapper; import java.io.ByteArrayOutputStream; @@ -50,8 +54,8 @@ public RestEmitter(RestEmitterConfig config) { } this.httpClient = this.config.getAsyncHttpClientBuilder().build(); this.httpClient.start(); - this.ingestProposalUrl = this.config.getGmsUrl() + "/aspects?action=ingestProposal"; - this.configUrl = this.config.getGmsUrl() + "/config"; + this.ingestProposalUrl = this.config.getServer() + "/aspects?action=ingestProposal"; + this.configUrl = this.config.getServer() + "/config"; this.eventFormatter = this.config.getEventFormatter(); } diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitterConfig.java similarity index 95% rename from metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java rename to metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitterConfig.java index a5cf03956e3324..53a53f2bee2217 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/RestEmitterConfig.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitterConfig.java @@ -1,4 +1,4 @@ -package datahub.client; +package datahub.client.rest; import datahub.event.EventFormatter; import java.util.Collections; @@ -20,7 +20,7 @@ public class RestEmitterConfig { public static final String DEFAULT_AUTH_TOKEN = null; @Builder.Default - private final String gmsUrl = "http://localhost:8080"; + private final String server = "http://localhost:8080"; private final Integer timeoutSec; diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/client/rest/RestEmitterTest.java similarity index 96% rename from metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java rename to metadata-integration/java/datahub-client/src/test/java/datahub/client/rest/RestEmitterTest.java index 45d3ecf451ccbe..0809eead27c7d7 100644 --- a/metadata-integration/java/datahub-client/src/test/java/datahub/client/RestEmitterTest.java +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/client/rest/RestEmitterTest.java @@ -1,8 +1,12 @@ -package datahub.client; +package datahub.client.rest; import com.fasterxml.jackson.databind.ObjectMapper; import com.linkedin.dataset.DatasetProperties; import com.linkedin.events.metadata.ChangeType; +import datahub.client.Callback; +import datahub.client.MetadataWriteResponse; +import datahub.client.rest.RestEmitter; +import datahub.client.rest.RestEmitterConfig; import datahub.event.MetadataChangeProposalWrapper; import datahub.server.TestDataHubServer; import java.io.IOException; @@ -133,7 +137,7 @@ public void testExtraHeaders() throws Exception { public void mockServerTest() throws InterruptedException, ExecutionException, IOException { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.server("http://localhost:" + port)); Assert.assertTrue(emitter.testConnection()); } @@ -141,7 +145,7 @@ public void mockServerTest() throws InterruptedException, ExecutionException, IO public void multithreadedTestExecutors() throws Exception { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.server("http://localhost:" + port)); testDataHubServer.getMockServer() .when(request().withMethod("POST") @@ -218,7 +222,7 @@ private MetadataChangeProposalWrapper getMetadataChangeProposalWrapper(String de public void multithreadedTestSingleThreadCaller() throws Exception { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.server("http://localhost:" + port)); testDataHubServer.getMockServer() .when(request().withMethod("POST") @@ -276,7 +280,7 @@ public void multithreadedTestSingleThreadCaller() throws Exception { public void testCallback() throws Exception { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.server("http://localhost:" + port)); testDataHubServer.getMockServer() .when(request().withMethod("POST") @@ -311,7 +315,7 @@ public void onFailure(Throwable exception) { public void testTimeoutOnGet() { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.server("http://localhost:" + port)); testDataHubServer.getMockServer().reset(); testDataHubServer.getMockServer() @@ -339,7 +343,7 @@ public void testTimeoutOnGet() { public void testTimeoutOnGetWithTimeout() { TestDataHubServer testDataHubServer = new TestDataHubServer(); Integer port = testDataHubServer.getMockServer().getPort(); - RestEmitter emitter = RestEmitter.create(b -> b.gmsUrl("http://localhost:" + port)); + RestEmitter emitter = RestEmitter.create(b -> b.server("http://localhost:" + port)); testDataHubServer.getMockServer().reset(); testDataHubServer.getMockServer() diff --git a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java index 1394dc99c7215d..51be66ea3dfb28 100644 --- a/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java +++ b/spark-lineage/src/main/java/com/linkedin/datahub/lineage/consumer/impl/McpEmitter.java @@ -4,7 +4,7 @@ import com.linkedin.datahub.lineage.spark.model.LineageEvent; import com.linkedin.mxe.MetadataChangeProposal; import datahub.client.Emitter; -import datahub.client.RestEmitter; +import datahub.client.rest.RestEmitter; import java.io.IOException; import java.util.List; import java.util.Objects; @@ -53,7 +53,7 @@ private Emitter emitter() { if (conf.contains(GMS_URL_KEY)) { String gmsUrl = conf.get(GMS_URL_KEY); log.debug("REST emitter configured with GMS url " + gmsUrl); - return RestEmitter.create($ -> $.gmsUrl(gmsUrl)); + return RestEmitter.create($ -> $.server(gmsUrl)); } log.error("GMS URL not configured."); From 6d9082800229937550bbb21720cddd1b50981694 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Fri, 31 Dec 2021 20:03:27 +0530 Subject: [PATCH 22/24] fix checkstyle --- .../src/test/java/datahub/client/rest/RestEmitterTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/metadata-integration/java/datahub-client/src/test/java/datahub/client/rest/RestEmitterTest.java b/metadata-integration/java/datahub-client/src/test/java/datahub/client/rest/RestEmitterTest.java index 0809eead27c7d7..a770ce605a563e 100644 --- a/metadata-integration/java/datahub-client/src/test/java/datahub/client/rest/RestEmitterTest.java +++ b/metadata-integration/java/datahub-client/src/test/java/datahub/client/rest/RestEmitterTest.java @@ -5,8 +5,6 @@ import com.linkedin.events.metadata.ChangeType; import datahub.client.Callback; import datahub.client.MetadataWriteResponse; -import datahub.client.rest.RestEmitter; -import datahub.client.rest.RestEmitterConfig; import datahub.event.MetadataChangeProposalWrapper; import datahub.server.TestDataHubServer; import java.io.IOException; From e5bde3b481974c7784a1713ca039c1af5669dd97 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Sun, 2 Jan 2022 22:12:02 +0530 Subject: [PATCH 23/24] javadocs --- .../src/main/java/datahub/client/Emitter.java | 58 +++++++++++++- .../datahub/client/MetadataWriteResponse.java | 2 +- .../java/datahub/client/rest/RestEmitter.java | 77 +++++++++++++++---- 3 files changed, 121 insertions(+), 16 deletions(-) diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java index 24eec2df1a6ae9..4b6a4ad3dfebb1 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/Emitter.java @@ -6,14 +6,68 @@ import java.io.IOException; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import javax.annotation.Nonnull; +import javax.annotation.concurrent.ThreadSafe; +/** + * An interface implemented by all metadata emitters to DataHub. + * Typical usage: + * 1. Construct the emitter using the native constructor or builder for the Emitter. + * 2. Call `emitter.emit(mcpw, callback)` for each event you want to send + * 3. Wait for all events to be sent by inspecting the futures returned by each call or using callbacks + * 4. Call `emitter.close()` to finalize. + */ +@ThreadSafe public interface Emitter extends Closeable { - Future emit(MetadataChangeProposalWrapper mcpw, Callback callback) throws IOException; + /** + * Asynchronously emit a {@link MetadataChangeProposalWrapper} event. + * @param mcpw + * @param callback if not null, is called from the IO thread. Should be a quick operation. + * @return a {@link Future} for callers to inspect the result of the operation or block until one is available + * @throws IOException + */ + Future emit(@Nonnull MetadataChangeProposalWrapper mcpw, Callback callback) throws IOException; - Future emit(MetadataChangeProposal mcp, Callback callback) throws IOException; + /** + * Asynchronously emit a {@link MetadataChangeProposalWrapper} event. + * @param mcpw + * @return a {@link Future} for callers to inspect the result of the operation or block until one is available + * @throws IOException + */ + default Future emit(@Nonnull MetadataChangeProposalWrapper mcpw) throws IOException { + return emit(mcpw, null); + } + /** + * Asynchronously emit a {@link MetadataChangeProposal} event. Prefer using the sibling method + * that accepts a {@link MetadataChangeProposalWrapper} event as those are friendlier to construct. + * @param mcp + * @param callback if not null, is called from the IO thread. Should be a quick operation. + * @return a {@link Future} for callers to inspect the result of the operation or block until one is available + * @throws IOException + */ + Future emit(@Nonnull MetadataChangeProposal mcp, Callback callback) throws IOException; + + /** + * Asynchronously emit a {@link MetadataChangeProposal} event. Prefer using the sibling method + * that accepts a {@link MetadataChangeProposalWrapper} event as those are friendlier to construct. + * @param mcp + * @return a {@link Future} for callers to inspect the result of the operation or block until one is available + * @throws IOException + */ + default Future emit(@Nonnull MetadataChangeProposal mcp) throws IOException { + return emit(mcp, null); + } + + /** + * Test that the emitter can establish a valid connection to the DataHub platform + * @return true if a valid connection can be established, false or throws one of the exceptions otherwise + * @throws IOException + * @throws ExecutionException + * @throws InterruptedException + */ boolean testConnection() throws IOException, ExecutionException, InterruptedException; } diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java index 16aee6c0ff2748..969ef10c41a24f 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/MetadataWriteResponse.java @@ -18,7 +18,7 @@ public class MetadataWriteResponse { * If the write failed due to an exception thrown by the server * and we have access to it, then we store the stack trace here */ - String serverException; + String responseContent; /** * The underlying response object diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitter.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitter.java index 29ef68bbb692cc..221a00bd4376f1 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitter.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitter.java @@ -19,8 +19,10 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import javax.annotation.concurrent.ThreadSafe; import lombok.extern.slf4j.Slf4j; import org.apache.http.HttpResponse; +import org.apache.http.HttpStatus; import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpPost; @@ -30,7 +32,28 @@ import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +@ThreadSafe @Slf4j +/** + * The REST emitter is a thin wrapper on top of the Apache HttpClient + * (https://hc.apache.org/httpcomponents-client-4.5.x/index.html) library. It supports non-blocking emission of + * metadata and handles the details of JSON serialization of metadata aspects over the wire. + * + * Constructing a REST Emitter follows a lambda-based fluent builder pattern using the `create` method. + * e.g. + * RestEmitter emitter = RestEmitter.create(b -> b + * .server("http://localhost:8080") + * .extraHeaders(Collections.singletonMap("Custom-Header", "custom-val") + * ); + * You can also customize the underlying + * http client by calling the `customizeHttpAsyncClient` method on the builder. + * e.g. + * RestEmitter emitter = RestEmitter.create(b -> b + * .server("http://localhost:8080") + * .extraHeaders(Collections.singletonMap("Custom-Header", "custom-val") + * .customizeHttpAsyncClient(c -> c.setConnectionTimeToLive(30, TimeUnit.SECONDS)) + * ); + */ public class RestEmitter implements Emitter { private final RestEmitterConfig config; @@ -42,6 +65,10 @@ public class RestEmitter implements Emitter { private final CloseableHttpAsyncClient httpClient; private final EventFormatter eventFormatter; + /** + * The default constructor, prefer using the `create` factory method. + * @param config + */ public RestEmitter(RestEmitterConfig config) { this.config = config; // Override httpClient settings with RestEmitter configs if present @@ -62,34 +89,58 @@ public RestEmitter(RestEmitterConfig config) { private static MetadataWriteResponse mapResponse(HttpResponse response) { MetadataWriteResponse.MetadataWriteResponseBuilder builder = MetadataWriteResponse.builder().underlyingResponse(response); - if ((response != null) && (response.getStatusLine() != null) && (response.getStatusLine().getStatusCode() == 200 - || response.getStatusLine().getStatusCode() == 201)) { + if ((response != null) && (response.getStatusLine() != null) && (response.getStatusLine().getStatusCode() == HttpStatus.SC_OK + || response.getStatusLine().getStatusCode() == HttpStatus.SC_CREATED)) { builder.success(true); } else { builder.success(false); - try { - ByteArrayOutputStream result = new ByteArrayOutputStream(); - InputStream contentStream = response.getEntity().getContent(); - byte[] buffer = new byte[1024]; - int length = contentStream.read(buffer); - while (length > 0) { - result.write(buffer, 0, length); - length = contentStream.read(buffer); - } - builder.serverException(result.toString("UTF-8")); + } + // Read response content + try { + ByteArrayOutputStream result = new ByteArrayOutputStream(); + InputStream contentStream = response.getEntity().getContent(); + byte[] buffer = new byte[1024]; + int length = contentStream.read(buffer); + while (length > 0) { + result.write(buffer, 0, length); + length = contentStream.read(buffer); + } + builder.responseContent(result.toString("UTF-8")); } catch (Exception e) { // Catch all exceptions and still return a valid response object log.warn("Wasn't able to convert response into a string", e); } - } return builder.build(); } + + /** + * Constructing a REST Emitter follows a lambda-based fluent builder pattern using the `create` method. + * e.g. + * RestEmitter emitter = RestEmitter.create(b -> b + * .server("http://localhost:8080") // coordinates of gms server + * .extraHeaders(Collections.singletonMap("Custom-Header", "custom-val") + * ); + * You can also customize the underlying http client by calling the `customizeHttpAsyncClient` method on the builder. + * e.g. + * RestEmitter emitter = RestEmitter.create(b -> b + * .server("http://localhost:8080") + * .extraHeaders(Collections.singletonMap("Custom-Header", "custom-val") + * .customizeHttpAsyncClient(c -> c.setConnectionTimeToLive(30, TimeUnit.SECONDS)) + * ); + * @param builderSupplier + * @return a constructed RestEmitter. Call #testConnection to make sure this emitter has a valid connection to the server + */ public static RestEmitter create(Consumer builderSupplier) { RestEmitter restEmitter = new RestEmitter(RestEmitterConfig.builder().with(builderSupplier).build()); return restEmitter; } + /** + * Creates a RestEmitter with default settings. + * @return a constructed RestEmitter. + * Call #test_connection to validate that this emitter can communicate with the server. + */ public static RestEmitter createWithDefaults() { // No-op creator -> creates RestEmitter using default settings return create(b -> { From a73866aae09f34b03085e32b6c26bd93b0235087 Mon Sep 17 00:00:00 2001 From: Shirshanka Das Date: Sun, 2 Jan 2022 22:15:42 +0530 Subject: [PATCH 24/24] removing caCertifatePath, will implement in a future release --- .../src/main/java/datahub/client/rest/RestEmitterConfig.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitterConfig.java b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitterConfig.java index 53a53f2bee2217..b6a7e9b558a678 100644 --- a/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitterConfig.java +++ b/metadata-integration/java/datahub-client/src/main/java/datahub/client/rest/RestEmitterConfig.java @@ -31,8 +31,6 @@ public class RestEmitterConfig { @NonNull private final Map extraHeaders = Collections.EMPTY_MAP; - private final String caCertificatePath; - private final HttpAsyncClientBuilder asyncHttpClientBuilder; @Builder.Default