mirror of
https://github.com/datahub-project/datahub.git
synced 2025-08-15 12:46:53 +00:00
fix(ingest): spark-lineage - Adding additional debug logs to spark lineage (#5772)
This commit is contained in:
parent
c44fd626d8
commit
caec2ed235
@ -44,10 +44,13 @@ import datahub.spark.model.SQLQueryExecEndEvent;
|
|||||||
import datahub.spark.model.SQLQueryExecStartEvent;
|
import datahub.spark.model.SQLQueryExecStartEvent;
|
||||||
import datahub.spark.model.dataset.SparkDataset;
|
import datahub.spark.model.dataset.SparkDataset;
|
||||||
import lombok.extern.slf4j.Slf4j;
|
import lombok.extern.slf4j.Slf4j;
|
||||||
|
import org.apache.spark.util.JsonProtocol;
|
||||||
|
import org.json4s.jackson.JsonMethods$;
|
||||||
import scala.collection.JavaConversions;
|
import scala.collection.JavaConversions;
|
||||||
import scala.runtime.AbstractFunction1;
|
import scala.runtime.AbstractFunction1;
|
||||||
import scala.runtime.AbstractPartialFunction;
|
import scala.runtime.AbstractPartialFunction;
|
||||||
|
|
||||||
|
|
||||||
@Slf4j
|
@Slf4j
|
||||||
public class DatahubSparkListener extends SparkListener {
|
public class DatahubSparkListener extends SparkListener {
|
||||||
|
|
||||||
@ -58,7 +61,7 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
public static final String PIPELINE_PLATFORM_INSTANCE_KEY = PIPELINE_KEY + ".platformInstance";
|
public static final String PIPELINE_PLATFORM_INSTANCE_KEY = PIPELINE_KEY + ".platformInstance";
|
||||||
|
|
||||||
public static final String COALESCE_KEY = "coalesce_jobs";
|
public static final String COALESCE_KEY = "coalesce_jobs";
|
||||||
|
|
||||||
private final Map<String, AppStartEvent> appDetails = new ConcurrentHashMap<>();
|
private final Map<String, AppStartEvent> appDetails = new ConcurrentHashMap<>();
|
||||||
private final Map<String, Map<Long, SQLQueryExecStartEvent>> appSqlDetails = new ConcurrentHashMap<>();
|
private final Map<String, Map<Long, SQLQueryExecStartEvent>> appSqlDetails = new ConcurrentHashMap<>();
|
||||||
private final Map<String, McpEmitter> appEmitters = new ConcurrentHashMap<>();
|
private final Map<String, McpEmitter> appEmitters = new ConcurrentHashMap<>();
|
||||||
@ -78,12 +81,33 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
this.sqlStart = sqlStart;
|
this.sqlStart = sqlStart;
|
||||||
this.plan = plan;
|
this.plan = plan;
|
||||||
this.ctx = ctx;
|
this.ctx = ctx;
|
||||||
|
|
||||||
|
String jsonPlan = (plan != null) ? plan.toJSON() : null;
|
||||||
|
String sqlStartJson =
|
||||||
|
(sqlStart != null) ? JsonMethods$.MODULE$.compact(JsonProtocol.sparkEventToJson(sqlStart)) : null;
|
||||||
|
log.debug("SqlStartTask with parameters: sqlStart: {}, plan: {}, ctx: {}", sqlStartJson, jsonPlan, ctx);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void run() {
|
public void run() {
|
||||||
appSqlDetails.get(ctx.applicationId()).put(sqlStart.executionId(),
|
if (ctx == null) {
|
||||||
new SQLQueryExecStartEvent(ctx.conf().get("spark.master"), getPipelineName(ctx), ctx.applicationId(),
|
log.error("Context is null skipping run");
|
||||||
sqlStart.time(), sqlStart.executionId(), null));
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ctx.conf() == null) {
|
||||||
|
log.error("Context does not have config. Skipping run");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sqlStart == null) {
|
||||||
|
log.error("sqlStart is null skipping run");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
appSqlDetails.get(ctx.applicationId())
|
||||||
|
.put(sqlStart.executionId(),
|
||||||
|
new SQLQueryExecStartEvent(ctx.conf().get("spark.master"), getPipelineName(ctx), ctx.applicationId(),
|
||||||
|
sqlStart.time(), sqlStart.executionId(), null));
|
||||||
log.debug("PLAN for execution id: " + getPipelineName(ctx) + ":" + sqlStart.executionId() + "\n");
|
log.debug("PLAN for execution id: " + getPipelineName(ctx) + ":" + sqlStart.executionId() + "\n");
|
||||||
log.debug(plan.toString());
|
log.debug(plan.toString());
|
||||||
|
|
||||||
@ -94,8 +118,8 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
// Here assumption is that there will be only single target for single sql query
|
// Here assumption is that there will be only single target for single sql query
|
||||||
DatasetLineage lineage = new DatasetLineage(sqlStart.description(), plan.toString(),
|
DatasetLineage lineage =
|
||||||
outputDS.get().iterator().next());
|
new DatasetLineage(sqlStart.description(), plan.toString(), outputDS.get().iterator().next());
|
||||||
Collection<QueryPlan<?>> allInners = new ArrayList<>();
|
Collection<QueryPlan<?>> allInners = new ArrayList<>();
|
||||||
|
|
||||||
plan.collect(new AbstractPartialFunction<LogicalPlan, Void>() {
|
plan.collect(new AbstractPartialFunction<LogicalPlan, Void>() {
|
||||||
@ -140,8 +164,9 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
SQLQueryExecStartEvent evt = new SQLQueryExecStartEvent(ctx.conf().get("spark.master"), getPipelineName(ctx),
|
SQLQueryExecStartEvent evt =
|
||||||
ctx.applicationId(), sqlStart.time(), sqlStart.executionId(), lineage);
|
new SQLQueryExecStartEvent(ctx.conf().get("spark.master"), getPipelineName(ctx), ctx.applicationId(),
|
||||||
|
sqlStart.time(), sqlStart.executionId(), lineage);
|
||||||
|
|
||||||
appSqlDetails.get(ctx.applicationId()).put(sqlStart.executionId(), evt);
|
appSqlDetails.get(ctx.applicationId()).put(sqlStart.executionId(), evt);
|
||||||
|
|
||||||
@ -257,11 +282,13 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
public Void apply(SparkContext sc) {
|
public Void apply(SparkContext sc) {
|
||||||
SQLQueryExecStartEvent start = appSqlDetails.get(sc.applicationId()).remove(sqlEnd.executionId());
|
SQLQueryExecStartEvent start = appSqlDetails.get(sc.applicationId()).remove(sqlEnd.executionId());
|
||||||
if (start == null) {
|
if (start == null) {
|
||||||
log.error("Execution end event received, but start event missing for appId/sql exec Id " + sc.applicationId()
|
log.error(
|
||||||
+ ":" + sqlEnd.executionId());
|
"Execution end event received, but start event missing for appId/sql exec Id " + sc.applicationId() + ":"
|
||||||
|
+ sqlEnd.executionId());
|
||||||
} else if (start.getDatasetLineage() != null) {
|
} else if (start.getDatasetLineage() != null) {
|
||||||
SQLQueryExecEndEvent evt = new SQLQueryExecEndEvent(LineageUtils.getMaster(sc), sc.appName(),
|
SQLQueryExecEndEvent evt =
|
||||||
sc.applicationId(), sqlEnd.time(), sqlEnd.executionId(), start);
|
new SQLQueryExecEndEvent(LineageUtils.getMaster(sc), sc.appName(), sc.applicationId(), sqlEnd.time(),
|
||||||
|
sqlEnd.executionId(), start);
|
||||||
McpEmitter emitter = appEmitters.get(sc.applicationId());
|
McpEmitter emitter = appEmitters.get(sc.applicationId());
|
||||||
if (emitter != null) {
|
if (emitter != null) {
|
||||||
emitter.accept(evt);
|
emitter.accept(evt);
|
||||||
@ -271,7 +298,7 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized void checkOrCreateApplicationSetup(SparkContext ctx) {
|
private synchronized void checkOrCreateApplicationSetup(SparkContext ctx) {
|
||||||
ExecutorService pool = null;
|
ExecutorService pool = null;
|
||||||
String appId = ctx.applicationId();
|
String appId = ctx.applicationId();
|
||||||
@ -281,18 +308,17 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
appConfig.put(appId, datahubConf);
|
appConfig.put(appId, datahubConf);
|
||||||
Config pipelineConfig = datahubConf.hasPath(PIPELINE_KEY) ? datahubConf.getConfig(PIPELINE_KEY)
|
Config pipelineConfig = datahubConf.hasPath(PIPELINE_KEY) ? datahubConf.getConfig(PIPELINE_KEY)
|
||||||
: com.typesafe.config.ConfigFactory.empty();
|
: com.typesafe.config.ConfigFactory.empty();
|
||||||
AppStartEvent evt = new AppStartEvent(LineageUtils.getMaster(ctx), getPipelineName(ctx), appId, ctx.startTime(),
|
AppStartEvent evt =
|
||||||
ctx.sparkUser(), pipelineConfig);
|
new AppStartEvent(LineageUtils.getMaster(ctx), getPipelineName(ctx), appId, ctx.startTime(), ctx.sparkUser(),
|
||||||
|
pipelineConfig);
|
||||||
|
|
||||||
appEmitters.computeIfAbsent(appId,
|
appEmitters.computeIfAbsent(appId,
|
||||||
s -> datahubConf.hasPath(COALESCE_KEY) && datahubConf.getBoolean(COALESCE_KEY)
|
s -> datahubConf.hasPath(COALESCE_KEY) && datahubConf.getBoolean(COALESCE_KEY) ? new CoalesceJobsEmitter(
|
||||||
? new CoalesceJobsEmitter(datahubConf)
|
datahubConf) : new McpEmitter(datahubConf)).accept(evt);
|
||||||
: new McpEmitter(datahubConf))
|
|
||||||
.accept(evt);
|
|
||||||
consumers().forEach(c -> c.accept(evt));
|
consumers().forEach(c -> c.accept(evt));
|
||||||
appDetails.put(appId, evt);
|
appDetails.put(appId, evt);
|
||||||
appSqlDetails.put(appId, new ConcurrentHashMap<>());
|
appSqlDetails.put(appId, new ConcurrentHashMap<>());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private String getPipelineName(SparkContext cx) {
|
private String getPipelineName(SparkContext cx) {
|
||||||
@ -329,10 +355,11 @@ public class DatahubSparkListener extends SparkListener {
|
|||||||
if (conf.contains(CONSUMER_TYPE_KEY)) {
|
if (conf.contains(CONSUMER_TYPE_KEY)) {
|
||||||
String consumerTypes = conf.get(CONSUMER_TYPE_KEY);
|
String consumerTypes = conf.get(CONSUMER_TYPE_KEY);
|
||||||
return StreamSupport.stream(Splitter.on(",").trimResults().split(consumerTypes).spliterator(), false)
|
return StreamSupport.stream(Splitter.on(",").trimResults().split(consumerTypes).spliterator(), false)
|
||||||
.map(x -> LineageUtils.getConsumer(x)).filter(Objects::nonNull).collect(Collectors.toList());
|
.map(x -> LineageUtils.getConsumer(x))
|
||||||
|
.filter(Objects::nonNull)
|
||||||
|
.collect(Collectors.toList());
|
||||||
} else {
|
} else {
|
||||||
return Collections.emptyList();
|
return Collections.emptyList();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user