Skip to content

Commit 3316d40

Browse files
authored
fix(spark): Finegrained lineage is emitted on the DataJob and not on the emitted Datasets. (#11956)
1 parent 91c1c6b commit 3316d40

File tree

9 files changed

+319
-84
lines changed

9 files changed

+319
-84
lines changed

metadata-integration/java/acryl-spark-lineage/README.md

+58-43
Large diffs are not rendered by default.

metadata-integration/java/acryl-spark-lineage/src/main/java/datahub/spark/DatahubEventEmitter.java

+25-1
Original file line numberDiff line numberDiff line change
@@ -1,12 +1,18 @@
11
package datahub.spark;
22

3+
import static com.linkedin.metadata.Constants.*;
34
import static datahub.spark.converter.SparkStreamingEventToDatahub.*;
45
import static io.datahubproject.openlineage.converter.OpenLineageToDataHub.*;
56
import static io.datahubproject.openlineage.utils.DatahubUtils.*;
67

8+
import com.fasterxml.jackson.annotation.JsonInclude;
9+
import com.fasterxml.jackson.core.StreamReadConstraints;
10+
import com.fasterxml.jackson.databind.ObjectMapper;
711
import com.linkedin.common.GlobalTags;
812
import com.linkedin.common.UrnArray;
913
import com.linkedin.common.urn.DataJobUrn;
14+
import com.linkedin.data.DataMap;
15+
import com.linkedin.data.template.JacksonDataTemplateCodec;
1016
import com.linkedin.data.template.StringMap;
1117
import com.linkedin.dataprocess.DataProcessInstanceRelationships;
1218
import com.linkedin.dataprocess.RunResultType;
@@ -62,12 +68,23 @@ public class DatahubEventEmitter extends EventEmitter {
6268
private final Map<String, MetadataChangeProposalWrapper> schemaMap = new HashMap<>();
6369
private SparkLineageConf datahubConf;
6470
private static final int DEFAULT_TIMEOUT_SEC = 10;
71+
private final ObjectMapper objectMapper;
72+
private final JacksonDataTemplateCodec dataTemplateCodec;
6573

6674
private final EventFormatter eventFormatter = new EventFormatter();
6775

6876
public DatahubEventEmitter(SparkOpenLineageConfig config, String applicationJobName)
6977
throws URISyntaxException {
7078
super(config, applicationJobName);
79+
objectMapper = new ObjectMapper().setSerializationInclusion(JsonInclude.Include.NON_NULL);
80+
int maxSize =
81+
Integer.parseInt(
82+
System.getenv()
83+
.getOrDefault(INGESTION_MAX_SERIALIZED_STRING_LENGTH, MAX_JACKSON_STRING_SIZE));
84+
objectMapper
85+
.getFactory()
86+
.setStreamReadConstraints(StreamReadConstraints.builder().maxStringLength(maxSize).build());
87+
dataTemplateCodec = new JacksonDataTemplateCodec(objectMapper.getFactory());
7188
}
7289

7390
private Optional<Emitter> getEmitter() {
@@ -407,7 +424,14 @@ protected void emitMcps(List<MetadataChangeProposal> mcps) {
407424
.map(
408425
mcp -> {
409426
try {
410-
log.info("emitting mcpw: " + mcp);
427+
if (this.datahubConf.isLogMcps()) {
428+
DataMap map = mcp.data();
429+
String serializedMCP = dataTemplateCodec.mapToString(map);
430+
log.info("emitting mcpw: {}", serializedMCP);
431+
} else {
432+
log.info(
433+
"emitting aspect: {} for urn: {}", mcp.getAspectName(), mcp.getEntityUrn());
434+
}
411435
return emitter.get().emit(mcp);
412436
} catch (IOException ioException) {
413437
log.error("Failed to emit metadata to DataHub", ioException);

metadata-integration/java/acryl-spark-lineage/src/main/java/datahub/spark/conf/SparkConfigParser.java

+19
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ public class SparkConfigParser {
3131
public static final String FILE_EMITTER_FILE_NAME = "file.filename";
3232
public static final String DISABLE_SSL_VERIFICATION_KEY = "rest.disable_ssl_verification";
3333
public static final String REST_DISABLE_CHUNKED_ENCODING = "rest.disable_chunked_encoding";
34+
public static final String CONFIG_LOG_MCPS = "log.mcps";
3435

3536
public static final String MAX_RETRIES = "rest.max_retries";
3637
public static final String RETRY_INTERVAL_IN_SEC = "rest.retry_interval_in_sec";
@@ -51,6 +52,7 @@ public class SparkConfigParser {
5152

5253
public static final String COALESCE_KEY = "coalesce_jobs";
5354
public static final String PATCH_ENABLED = "patch.enabled";
55+
public static final String LEGACY_LINEAGE_CLEANUP = "legacyLineageCleanup.enabled";
5456
public static final String DISABLE_SYMLINK_RESOLUTION = "disableSymlinkResolution";
5557

5658
public static final String STAGE_METADATA_COALESCING = "stage_metadata_coalescing";
@@ -158,6 +160,7 @@ public static DatahubOpenlineageConfig sparkConfigToDatahubOpenlineageConf(
158160
Config sparkConfig, SparkAppContext sparkAppContext) {
159161
DatahubOpenlineageConfig.DatahubOpenlineageConfigBuilder builder =
160162
DatahubOpenlineageConfig.builder();
163+
builder.isSpark(true);
161164
builder.filePartitionRegexpPattern(
162165
SparkConfigParser.getFilePartitionRegexpPattern(sparkConfig));
163166
builder.fabricType(SparkConfigParser.getCommonFabricType(sparkConfig));
@@ -172,6 +175,7 @@ public static DatahubOpenlineageConfig sparkConfigToDatahubOpenlineageConf(
172175
builder.commonDatasetPlatformInstance(SparkConfigParser.getCommonPlatformInstance(sparkConfig));
173176
builder.hivePlatformAlias(SparkConfigParser.getHivePlatformAlias(sparkConfig));
174177
builder.usePatch(SparkConfigParser.isPatchEnabled(sparkConfig));
178+
builder.removeLegacyLineage(SparkConfigParser.isLegacyLineageCleanupEnabled(sparkConfig));
175179
builder.disableSymlinkResolution(SparkConfigParser.isDisableSymlinkResolution(sparkConfig));
176180
builder.lowerCaseDatasetUrns(SparkConfigParser.isLowerCaseDatasetUrns(sparkConfig));
177181
try {
@@ -311,6 +315,13 @@ public static boolean isDatasetMaterialize(Config datahubConfig) {
311315
&& datahubConfig.getBoolean(DATASET_MATERIALIZE_KEY);
312316
}
313317

318+
public static boolean isLogMcps(Config datahubConfig) {
319+
if (datahubConfig.hasPath(CONFIG_LOG_MCPS)) {
320+
return datahubConfig.getBoolean(CONFIG_LOG_MCPS);
321+
}
322+
return true;
323+
}
324+
314325
public static boolean isIncludeSchemaMetadata(Config datahubConfig) {
315326
if (datahubConfig.hasPath(DATASET_INCLUDE_SCHEMA_METADATA)) {
316327
return datahubConfig.getBoolean(DATASET_INCLUDE_SCHEMA_METADATA);
@@ -352,6 +363,14 @@ public static boolean isPatchEnabled(Config datahubConfig) {
352363
return datahubConfig.hasPath(PATCH_ENABLED) && datahubConfig.getBoolean(PATCH_ENABLED);
353364
}
354365

366+
public static boolean isLegacyLineageCleanupEnabled(Config datahubConfig) {
367+
if (!datahubConfig.hasPath(LEGACY_LINEAGE_CLEANUP)) {
368+
return false;
369+
}
370+
return datahubConfig.hasPath(LEGACY_LINEAGE_CLEANUP)
371+
&& datahubConfig.getBoolean(LEGACY_LINEAGE_CLEANUP);
372+
}
373+
355374
public static boolean isDisableSymlinkResolution(Config datahubConfig) {
356375
if (!datahubConfig.hasPath(DISABLE_SYMLINK_RESOLUTION)) {
357376
return false;

metadata-integration/java/acryl-spark-lineage/src/main/java/datahub/spark/conf/SparkLineageConf.java

+2
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ public class SparkLineageConf {
1717
final DatahubOpenlineageConfig openLineageConf;
1818
@Builder.Default final boolean coalesceEnabled = true;
1919
@Builder.Default final boolean emitCoalescePeriodically = false;
20+
@Builder.Default final boolean logMcps = true;
2021
final SparkAppContext sparkAppContext;
2122
final DatahubEmitterConfig datahubEmitterConfig;
2223
@Builder.Default final List<String> tags = new LinkedList<>();
@@ -32,6 +33,7 @@ public static SparkLineageConf toSparkLineageConf(
3233
SparkConfigParser.sparkConfigToDatahubOpenlineageConf(sparkConfig, sparkAppContext);
3334
builder.openLineageConf(datahubOpenlineageConfig);
3435
builder.coalesceEnabled(SparkConfigParser.isCoalesceEnabled(sparkConfig));
36+
builder.logMcps(SparkConfigParser.isLogMcps(sparkConfig));
3537
if (SparkConfigParser.getTags(sparkConfig) != null) {
3638
builder.tags(Arrays.asList(Objects.requireNonNull(SparkConfigParser.getTags(sparkConfig))));
3739
}

metadata-integration/java/acryl-spark-lineage/src/test/java/datahub/spark/OpenLineageEventToDatahubTest.java

+28
Original file line numberDiff line numberDiff line change
@@ -814,4 +814,32 @@ public void testProcessGCSInputsOutputs() throws URISyntaxException, IOException
814814
dataset.getUrn().toString());
815815
}
816816
}
817+
818+
public void testProcessMappartitionJob() throws URISyntaxException, IOException {
819+
DatahubOpenlineageConfig.DatahubOpenlineageConfigBuilder builder =
820+
DatahubOpenlineageConfig.builder();
821+
builder.fabricType(FabricType.DEV);
822+
builder.lowerCaseDatasetUrns(true);
823+
builder.materializeDataset(true);
824+
builder.includeSchemaMetadata(true);
825+
builder.isSpark(true);
826+
827+
String olEvent =
828+
IOUtils.toString(
829+
this.getClass().getResourceAsStream("/ol_events/map_partition_job.json"),
830+
StandardCharsets.UTF_8);
831+
832+
OpenLineage.RunEvent runEvent = OpenLineageClientUtils.runEventFromJson(olEvent);
833+
DatahubJob datahubJob = OpenLineageToDataHub.convertRunEventToJob(runEvent, builder.build());
834+
835+
assertNotNull(datahubJob);
836+
837+
assertEquals(1, datahubJob.getInSet().size());
838+
for (DatahubDataset dataset : datahubJob.getInSet()) {
839+
assertEquals(
840+
"urn:li:dataset:(urn:li:dataPlatform:s3,my-bucket/my_dir/my_file.csv,DEV)",
841+
dataset.getUrn().toString());
842+
}
843+
assertEquals(0, datahubJob.getOutSet().size());
844+
}
817845
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,66 @@
1+
{
2+
"eventTime": "2024-11-20T12:59:29.059Z",
3+
"producer": "https://github.com/OpenLineage/OpenLineage/tree/1.24.2/integration/spark",
4+
"schemaURL": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunEvent",
5+
"eventType": "START",
6+
"run": {
7+
"runId": "01902a1e-0b05-750e-b38d-439998f7a853",
8+
"facets": {
9+
"parent": {
10+
"_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.24.2/integration/spark",
11+
"_schemaURL": "https://openlineage.io/spec/facets/1-0-1/ParentRunFacet.json#/$defs/ParentRunFacet",
12+
"run": {
13+
"runId": "01902a1e-0b05-750e-b38d-439998f7a853"
14+
},
15+
"job": {
16+
"namespace": "default",
17+
"name": "spark_context_session"
18+
}
19+
},
20+
"processing_engine": {
21+
"_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.24.2/integration/spark",
22+
"_schemaURL": "https://openlineage.io/spec/facets/1-1-1/ProcessingEngineRunFacet.json#/$defs/ProcessingEngineRunFacet",
23+
"version": "3.4.2",
24+
"name": "spark"
25+
},
26+
"spark_jobDetails": {
27+
"_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.24.2/integration/spark",
28+
"_schemaURL": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunFacet",
29+
"jobId": 0
30+
},
31+
"spark_properties": {
32+
"_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.24.2/integration/spark",
33+
"_schemaURL": "https://openlineage.io/spec/2-0-2/OpenLineage.json#/$defs/RunFacet",
34+
"properties": {
35+
"spark.master": "yarn",
36+
"spark.app.name": "SparkContextSession"
37+
}
38+
}
39+
}
40+
},
41+
"job": {
42+
"namespace": "default",
43+
"name": "spark_context_session.map_partitions_parallel_collection",
44+
"facets": {
45+
"jobType": {
46+
"_producer": "https://github.com/OpenLineage/OpenLineage/tree/1.24.2/integration/spark",
47+
"_schemaURL": "https://openlineage.io/spec/facets/2-0-3/JobTypeJobFacet.json#/$defs/JobTypeJobFacet",
48+
"processingType": "BATCH",
49+
"integration": "SPARK",
50+
"jobType": "RDD_JOB"
51+
}
52+
}
53+
},
54+
"inputs": [
55+
{
56+
"namespace": "s3://my-bucket",
57+
"name": "my_dir/my_file.csv"
58+
}
59+
],
60+
"outputs": [
61+
{
62+
"namespace": "s3://my-bucket",
63+
"name": "my_dir/my_file.csv"
64+
}
65+
]
66+
}

metadata-integration/java/openlineage-converter/src/main/java/io/datahubproject/openlineage/config/DatahubOpenlineageConfig.java

+2
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
@Getter
1717
@ToString
1818
public class DatahubOpenlineageConfig {
19+
@Builder.Default private final boolean isSpark = false;
1920
@Builder.Default private final boolean isStreaming = false;
2021
@Builder.Default private final String pipelineName = null;
2122
private final String platformInstance;
@@ -34,6 +35,7 @@ public class DatahubOpenlineageConfig {
3435
@Builder.Default private Map<String, String> urnAliases = new HashMap<>();
3536
@Builder.Default private final boolean disableSymlinkResolution = false;
3637
@Builder.Default private final boolean lowerCaseDatasetUrns = false;
38+
@Builder.Default private final boolean removeLegacyLineage = false;
3739

3840
public List<PathSpec> getPathSpecsForPlatform(String platform) {
3941
if ((pathSpecs == null) || (pathSpecs.isEmpty())) {

metadata-integration/java/openlineage-converter/src/main/java/io/datahubproject/openlineage/converter/OpenLineageToDataHub.java

+22-1
Original file line numberDiff line numberDiff line change
@@ -675,9 +675,30 @@ private static void convertJobToDataJob(
675675
datahubJob.setJobInfo(dji);
676676
DataJobInputOutput inputOutput = new DataJobInputOutput();
677677

678+
boolean inputsEqualOutputs = false;
679+
if ((datahubConf.isSpark())
680+
&& ((event.getInputs() != null && event.getOutputs() != null)
681+
&& (event.getInputs().size() == event.getOutputs().size()))) {
682+
inputsEqualOutputs =
683+
event.getInputs().stream()
684+
.map(OpenLineage.Dataset::getName)
685+
.collect(Collectors.toSet())
686+
.equals(
687+
event.getOutputs().stream()
688+
.map(OpenLineage.Dataset::getName)
689+
.collect(Collectors.toSet()));
690+
if (inputsEqualOutputs) {
691+
log.info(
692+
"Inputs equals Outputs: {}. This is most probably because of an rdd map operation and we only process Inputs",
693+
inputsEqualOutputs);
694+
}
695+
}
696+
678697
processJobInputs(datahubJob, event, datahubConf);
679698

680-
processJobOutputs(datahubJob, event, datahubConf);
699+
if (!inputsEqualOutputs) {
700+
processJobOutputs(datahubJob, event, datahubConf);
701+
}
681702

682703
DataProcessInstanceRunEvent dpire = processDataProcessInstanceResult(event);
683704
datahubJob.setDataProcessInstanceRunEvent(dpire);

0 commit comments

Comments
 (0)