Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Streaming support #6

Open
wants to merge 7 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ public final class JobDagActions {
private static final String ACTION_TARGET = "action_target";

@Getter
private final Queue<IJobDagAction> actions;
private Queue<IJobDagAction> actions;
private final Reporters reporters;

@Getter
Expand All @@ -78,6 +78,10 @@ public JobDagActions(@NonNull final Reporters reporters, @NotEmpty final String
this.reporters = reporters;
}

public void reset() {
this.actions = new ConcurrentLinkedDeque<>();
}

/**
* Add an action to the container
* @param action IAction to hold
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@ public class ReporterAction implements IJobDagAction {
@Getter
private final Reporters reporters;
@Getter
private final JobMetrics jobMetrics;
private final DataFeedMetrics dataFeedMetrics;
private JobMetrics jobMetrics;
private DataFeedMetrics dataFeedMetrics;

@Override
public boolean execute(final boolean success) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
*/
package com.uber.marmaray.common.configuration;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
Expand Down Expand Up @@ -117,6 +118,19 @@ public void loadYamlStream(@NonNull final InputStream yamlStream,
}
}

public void loadJSONString(@NonNull final String jsonString, @NonNull final Optional<String> scope) {
try {
final ObjectMapper mapper = new ObjectMapper();
final JsonNode jsonNode = mapper.readTree(jsonString);
final JsonNode scopeOverriddenJsonNode = handleScopeOverriding(scope, jsonNode);
parseConfigJson(scopeOverriddenJsonNode, "");
} catch (IOException e) {
final String errorMsg = "Error loading config from string";
log.error(errorMsg, e);
throw new JobRuntimeException(errorMsg, e);
}
}

private JsonNode handleScopeOverriding(
@NonNull final Optional<String> scope, @NonNull final JsonNode jsonNode) {
return new ConfigScopeResolver(SCOPE_OVERRIDE_MAPPING_KEY)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
package com.uber.marmaray.common.configuration;

import lombok.Getter;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;

@Slf4j
public class StreamingConfiguration {
public static final String STREAMING_PREFIX = Configuration.MARMARAY_PREFIX + "streaming.";

/**
* Flag to control whether error table is enabled
*/
public static final String IS_ENABLED = STREAMING_PREFIX + "enabled";
public static final boolean DEFAULT_IS_ENABLED = false;

@Getter
private final Configuration conf;

@Getter
public final boolean isEnabled;

public StreamingConfiguration(@NonNull final Configuration conf) {
this.conf = conf;
this.isEnabled = conf.getBooleanProperty(IS_ENABLED, DEFAULT_IS_ENABLED);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,4 +26,5 @@ public Dag(@NonNull final String jobName, @NonNull final String dataFeedName) {

public abstract IStatus execute();

public abstract void clean();
}
Original file line number Diff line number Diff line change
Expand Up @@ -211,6 +211,14 @@ public IStatus execute() {
return status;
}

@Override
public void clean() {
this.sinkDag.clean();
this.dataFeedMetrics.clean();
this.jobMetrics.clean();
this.postJobDagActions.reset();
}

private void reportStatus(final boolean successful) {
final long statusValue =
successful ? DataFeedMetricNames.RESULT_SUCCESS : DataFeedMetricNames.RESULT_FAILURE;
Expand Down
184 changes: 110 additions & 74 deletions marmaray/src/main/java/com/uber/marmaray/common/job/JobManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import com.uber.marmaray.common.actions.IJobDagAction;
import com.uber.marmaray.common.actions.JobDagActions;
import com.uber.marmaray.common.configuration.Configuration;
import com.uber.marmaray.common.configuration.StreamingConfiguration;
import com.uber.marmaray.common.exceptions.JobRuntimeException;
import com.uber.marmaray.common.exceptions.MetadataException;
import com.uber.marmaray.common.metadata.JobManagerMetadataTracker;
Expand Down Expand Up @@ -181,81 +182,84 @@ public static JobManager createJobManager(@NonNull final Configuration conf,
/**
* Execute all registered {@link JobDag}, then perform all registered {@link IJobDagAction}
*/
public void run() {
final Queue<Future<Pair<String, IStatus>>> futures = new ConcurrentLinkedDeque<>();
final AtomicBoolean isSuccess = new AtomicBoolean(true);
// ensure the SparkContext has been created
Preconditions.checkState(!this.jobDags.isEmpty(), "No job dags to execute");
final JavaSparkContext javaSparkContext = sparkFactory.getSparkContext();
TimeoutManager.init(this.conf, javaSparkContext.sc());
final boolean hasMultipleDags = this.jobDags.size() > 1;
final Queue<Dag> runtimeJobDagOrder;
if (hasMultipleDags && this.jobExecutionStrategy.isPresent()) {
runtimeJobDagOrder = new ConcurrentLinkedDeque<>(this.jobExecutionStrategy.get().sort(this.jobDags));
} else {
runtimeJobDagOrder = this.jobDags;
}
try {
ThreadPoolService.init(this.conf);
runtimeJobDagOrder.forEach(jobDag ->
futures.add(ThreadPoolService.submit(
() -> {
SparkJobTracker.setJobName(javaSparkContext.sc(), jobDag.getDataFeedName());
if (hasMultipleDags) {
setSparkStageName(javaSparkContext, jobDag.getDataFeedName());
public void run(@NonNull final StreamingConfiguration streamingConf) {
do {
final Queue<Future<Pair<String, IStatus>>> futures = new ConcurrentLinkedDeque<>();
final AtomicBoolean isSuccess = new AtomicBoolean(true);
// ensure the SparkContext has been created
Preconditions.checkState(!this.jobDags.isEmpty(), "No job dags to execute");
final JavaSparkContext javaSparkContext = sparkFactory.getSparkContext();
TimeoutManager.init(this.conf, javaSparkContext.sc());
final boolean hasMultipleDags = this.jobDags.size() > 1;
final Queue<Dag> runtimeJobDagOrder;
if (hasMultipleDags && this.jobExecutionStrategy.isPresent()) {
runtimeJobDagOrder = new ConcurrentLinkedDeque<>(this.jobExecutionStrategy.get().sort(this.jobDags));
} else {
runtimeJobDagOrder = this.jobDags;
}
try {
ThreadPoolService.init(this.conf);
runtimeJobDagOrder.forEach(jobDag ->
futures.add(ThreadPoolService.submit(
() -> {
SparkJobTracker.setJobName(javaSparkContext.sc(), jobDag.getDataFeedName());
if (hasMultipleDags) {
setSparkStageName(javaSparkContext, jobDag.getDataFeedName());
}
final IStatus status = jobDag.execute();
this.jobManagerStatus.addJobStatus(jobDag.getJobName(), status);
return new Pair<>(jobDag.getJobName(), status);
}, ThreadPoolServiceTier.JOB_DAG_TIER)));

TimeoutManager.getInstance().startMonitorThread();
futures.forEach(future -> {
try {
final Optional<Pair<String, IStatus>> result = Optional.fromNullable(future.get());
IStatus.Status status = result.get().value().getStatus();
log.info("job dag, name: {}, status: {}",
result.get().key(), status.name());
if (IStatus.Status.FAILURE.equals(status)) {
log.error("Unsuccessful run, jobdag: {}", result.get().key());
isSuccess.set(false);
}
} catch (Exception e) {
log.error("Error running job", e);
isSuccess.set(false);
this.jobManagerStatus.setStatus(IStatus.Status.FAILURE);
this.jobManagerStatus.addException(e);
}
final IStatus status = jobDag.execute();
this.jobManagerStatus.addJobStatus(jobDag.getJobName(), status);
return new Pair<>(jobDag.getJobName(), status);
}, ThreadPoolServiceTier.JOB_DAG_TIER)));

TimeoutManager.getInstance().startMonitorThread();
futures.forEach(future -> {
try {
final Optional<Pair<String, IStatus>> result = Optional.fromNullable(future.get());
IStatus.Status status = result.get().value().getStatus();
log.info("job dag, name: {}, status: {}",
result.get().key(), status.name());
if (IStatus.Status.FAILURE.equals(status)) {
log.error("Unsuccessful run, jobdag: {}", result.get().key());
isSuccess.set(false);
}
} catch (Exception e) {
log.error("Error running job", e);
isSuccess.set(false);
this.jobManagerStatus.setStatus(IStatus.Status.FAILURE);
this.jobManagerStatus.addException(e);
}
);
if (TimeoutManager.getInstance().getTimedOut()) {
log.error("Time out error while running job.");
isSuccess.set(false);
}
// if we're not reporting success/failure through status, we need to throw an exception on failure
if (!isSuccess.get()) {
throw new JobRuntimeException("Error while running job. Look at previous log entries for detail");
}
);
if (TimeoutManager.getInstance().getTimedOut()) {
log.error("Time out error while running job.");
} catch (final Throwable t) {
log.error("Failed in JobManager", t);
isSuccess.set(false);
this.jobManagerStatus.setStatus(IStatus.Status.FAILURE);
if (t instanceof Exception) {
// trap exceptions and add them to the status
this.jobManagerStatus.addException((Exception) t);
} else {
// let errors be thrown
throw t;
}
} finally {
this.postJobManagerActions.execute(isSuccess.get());
resetManager(!isSuccess.get());
this.reporters.getReporters().forEach(IReporter::finish);
}
// if we're not reporting success/failure through status, we need to throw an exception on failure
if (!isSuccess.get()) {
throw new JobRuntimeException("Error while running job. Look at previous log entries for detail");
}
} catch (final Throwable t) {
log.error("Failed in JobManager", t);
isSuccess.set(false);
this.jobManagerStatus.setStatus(IStatus.Status.FAILURE);
if (t instanceof Exception) {
// trap exceptions and add them to the status
this.jobManagerStatus.addException((Exception) t);
} else {
// let errors be thrown
throw t;
}
} finally {
this.postJobManagerActions.execute(isSuccess.get());
shutdown(!isSuccess.get());
this.reporters.getReporters().forEach(IReporter::finish);
}
} while (streamingConf.isEnabled);
this.shutdown(false);
}

/**
* Add {@link JobDag} to be executed on {@link #run()}
* Add {@link JobDag} to be executed on
* @param jobDag JobDag to be added
*/
public void addJobDag(@NonNull final Dag jobDag) {
Expand All @@ -267,7 +271,19 @@ public void addJobDag(@NonNull final Dag jobDag) {
}

/**
* Add collection of {@link JobDag} to be executed on {@link #run()}
* Remove {@link JobDag} to be executed on
* @param jobDag JobDag to be added
*/
public void removeJobDag(@NonNull final Dag jobDag) {
if (jobLockManager.lockDag(jobDag.getJobName(), jobDag.getDataFeedName())) {
this.jobDags.remove(jobDag);
} else {
log.warn("Failed to obtain lock for JobDag {} - {}", jobDag.getJobName(), jobDag.getDataFeedName());
}
}

/**
* Add collection of {@link JobDag} to be executed on
* @param jobDags collection of JobDags to be added
*/
public void addJobDags(@NonNull final Collection<? extends JobDag> jobDags) {
Expand Down Expand Up @@ -300,19 +316,29 @@ public static void reset() {
}
}

private void shutdown(final boolean forceShutdown) {
ThreadPoolService.shutdown(forceShutdown);
private void resetManager(final boolean forceReset) {
ThreadPoolService.shutdown(forceReset);
if (this.isJobManagerMetadataEnabled()) {
this.jobDags.forEach(jobDag -> this.getTracker().set(jobDag.getDataFeedName(),
jobDag.getJobManagerMetadata()));
jobDag.getJobManagerMetadata()));
try {
this.getTracker().writeJobManagerMetadata();
} catch (MetadataException e) {
log.error("Unable to save metadata: {}", e.getMessage());
}
}
this.sparkFactory.stop();

// Cleanup metrics
this.jobDags.forEach(dag -> dag.clean());
this.postJobManagerActions.reset();
this.jobLockManager.reset();
this.jobMetrics.clean();
}

public void shutdown(final boolean forceShutdown) {
this.resetManager(forceShutdown);
this.jobLockManager.stop();
this.sparkFactory.stop();
}

private static void setSparkStageName(@NonNull final JavaSparkContext jsc, @NotEmpty final String dataFeedName) {
Expand All @@ -335,9 +361,9 @@ private final class JobLockManager {
private final String jobFrequency;

@NonNull
private final TimerMetric managerTimerMetric;
private TimerMetric managerTimerMetric;
@NonNull
private final HashMap<String, TimerMetric> dagTimerMetricMap;
private HashMap<String, TimerMetric> dagTimerMetricMap;

private JobLockManager(@NonNull final Configuration conf, @NotEmpty final String frequency,
final boolean shouldLockFrequency) {
Expand Down Expand Up @@ -381,8 +407,18 @@ private boolean lockDag(@NotEmpty final String jobDagName, @NotEmpty final Strin
private void stop() {
log.info("Closing the LockManager in the JobManager.");
this.lockManager.close();
this.reset();
}

private void reset() {
managerTimerMetric.stop();
reporters.report(managerTimerMetric);
this.managerTimerMetric = new TimerMetric(JobMetricNames.JOB_MANAGER_LOCK_TIME_MS,
ImmutableMap.of(JOB_FREQUENCY_TAG, jobFrequency,
JOB_NAME_TAG, appName));

dagTimerMetricMap.forEach((dagName, timerMetric) -> reporters.report(timerMetric));
this.dagTimerMetricMap = new HashMap<>();
}
}
}
10 changes: 10 additions & 0 deletions marmaray/src/main/java/com/uber/marmaray/common/job/JobSubDag.java
Original file line number Diff line number Diff line change
Expand Up @@ -127,6 +127,16 @@ public void setDataFeedMetrics(@NonNull final DataFeedMetrics dataFeedMetrics) {
protected void commitNode() {
}

public void clean() {
if (this.dataFeedMetrics.isPresent()) {
this.dataFeedMetrics.get().clean();
}

if (this.jobMetrics.isPresent()) {
this.jobMetrics.get().clean();
}
}

/**
* Called to retrieve payload for child dag.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ public AtomicBoolean shouldSaveChanges() {
* commit and will save metadata information in it.
*/
public void saveChanges() {
if (!this.saveChanges.compareAndSet(true, false)) {
if (!this.saveChanges.get()) {
log.info("Metadata info is already saved. Not saving it again.");
return;
}
Expand Down
Loading