Skip to content
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 @@ -184,6 +184,7 @@ public class ConfigurationKeys {
public static final String FLOW_UNSCHEDULE_KEY = "flow.unschedule";
public static final String FLOW_OWNING_GROUP_KEY = "flow.owningGroup";
public static final String FLOW_SPEC_EXECUTOR = "flow.edge.specExecutors";
public static final String RM_HOST_KEY = "hadoop.resource.manager.rpc";

/**
* Common topology configuration properties.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ public static class FlowEventConstants {
public static final String JOB_TAG_FIELD = "jobTag";
public static final String JOB_EXECUTION_ID_FIELD = "jobExecutionId";
public static final String SPEC_EXECUTOR_FIELD = "specExecutor";
public static final String RM_HOST_FIELD = "rmHost";
public static final String LOW_WATERMARK_FIELD = "lowWatermark";
public static final String HIGH_WATERMARK_FIELD = "highWatermark";
public static final String PROCESSED_COUNT_FIELD = "processedCount";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@ public class ServiceMetricNames {
public static final String DATA_QUALITY_NON_EVALUATED_FILE_COUNT = "dataQualityNonEvaluatedFileCount";
public static final String DATA_QUALITY_BYTES_READ = "dataQualityBytesRead";
public static final String DATA_QUALITY_BYTES_WRITTEN = "dataQualityBytesWritten";
// RM metric names
public static final String RM_JOB_OBSERVED_COUNT = "rmJobObservedCount";

// Flow Compilation Meters and Timer
public static final String FLOW_COMPILATION_SUCCESSFUL_METER = GOBBLIN_SERVICE_PREFIX_WITH_DELIMITER + "flowCompilation.successful";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;

import io.opentelemetry.api.common.Attributes;
import io.opentelemetry.api.metrics.Meter;
import lombok.Data;
import lombok.extern.slf4j.Slf4j;

Expand All @@ -50,6 +52,9 @@
import org.apache.gobblin.converter.initializer.ConverterInitializerFactory;
import org.apache.gobblin.initializer.Initializer;
import org.apache.gobblin.destination.DestinationDatasetHandlerService;
import org.apache.gobblin.metrics.OpenTelemetryMetrics;
import org.apache.gobblin.metrics.OpenTelemetryMetricsBase;
import org.apache.gobblin.metrics.ServiceMetricNames;
import org.apache.gobblin.metrics.event.EventSubmitter;
import org.apache.gobblin.metrics.event.TimingEvent;
import org.apache.gobblin.runtime.AbstractJobLauncher;
Expand All @@ -75,6 +80,8 @@
import org.apache.gobblin.writer.initializer.WriterInitializer;
import org.apache.gobblin.writer.initializer.WriterInitializerFactory;

import static org.apache.gobblin.runtime.JobState.GAAS_OBSERVABILITY_METRICS_GROUPNAME;


@Slf4j
public class GenerateWorkUnitsImpl implements GenerateWorkUnits {
Expand Down Expand Up @@ -136,6 +143,8 @@ public GenerateWorkUnitsResult generateWorkUnits(Properties jobProps, EventSubmi
// TODO: provide for job cancellation (unless handling at the temporal-level of parent workflows)!
JobState jobState = new JobState(jobProps);
log.info("Created jobState: {}", jobState.toJsonString(true));
// emit jobs observed at RM level
emitMetrics(jobState);

int heartBeatInterval = JobStateUtils.getHeartBeatInterval(jobState);
heartBeatExecutor.scheduleAtFixedRate(() -> activityExecutionContext.heartbeat("Running GenerateWorkUnits"),
Expand Down Expand Up @@ -335,4 +344,42 @@ protected static EventTimer createWorkPreparedSizeDistillationTimer(
public static int getConfiguredNumSizeSummaryQuantiles(State state) {
return state.getPropAsInt(GenerateWorkUnits.NUM_WORK_UNITS_SIZE_SUMMARY_QUANTILES, GenerateWorkUnits.DEFAULT_NUM_WORK_UNITS_SIZE_SUMMARY_QUANTILES);
}

/**
* Emit metrics to indicate jobs observed at RM level
* @param jobState job state
*/
private void emitMetrics(JobState jobState) {
try {
OpenTelemetryMetricsBase otelMetrics = OpenTelemetryMetrics.getInstance(jobState);
if (otelMetrics == null) {
log.warn("OpenTelemetry metrics instance is null, skipping metrics emission");
return;
}

Meter meter = otelMetrics.getMeter(GAAS_OBSERVABILITY_METRICS_GROUPNAME);
Attributes tags = getEventAttributes(jobState);
log.info("Emitting metrics for job: {}", jobState.getJobName());
String jobMetricDescription = "Number of Jobs observed on RM";
String jobMetricName = ServiceMetricNames.RM_JOB_OBSERVED_COUNT;
meter.counterBuilder(jobMetricName).setDescription(jobMetricDescription).build().add(1, tags);
} catch (Exception e) {
log.error("Error in emitMetrics for job: {}", jobState.getJobName(), e);
}
}

private Attributes getEventAttributes(JobState jobState) {
Properties jobProperties = jobState.getProperties();

return Attributes.builder()
.put(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD, jobProperties.getProperty(ConfigurationKeys.FLOW_NAME_KEY, "NA"))
.put(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD,
jobProperties.getProperty(ConfigurationKeys.FLOW_GROUP_KEY, "NA"))
.put(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD,
jobProperties.getProperty(ConfigurationKeys.FLOW_EXECUTION_ID_KEY, "NA"))
.put(TimingEvent.FlowEventConstants.FLOW_FABRIC,
jobProperties.getProperty(ConfigurationKeys.METRICS_REPORTING_OPENTELEMETRY_FABRIC, "NA"))
.put(TimingEvent.FlowEventConstants.RM_HOST_FIELD, jobProperties.getProperty(ConfigurationKeys.RM_HOST_KEY, "NA"))
.build();
}
}