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

Cassandra 19480 Additional task execution specific instrumentation of job stats #51

Open
wants to merge 6 commits into
base: trunk
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
@@ -0,0 +1,47 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.cassandra.spark;

import java.util.Map;

/**
* Container class for the job metadata associated with a spark listener event.
*/
public class JobEventDetail
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we add a javadoc here?

{
private String internalJobID;
private Map<String, String> jobStats;

public JobEventDetail(String internalJobID, Map<String, String> jobStats)
{
this.internalJobID = internalJobID;
this.jobStats = jobStats;
}

public String internalJobID()
{
return internalJobID;
}

public Map<String, String> jobStats()
{
return jobStats;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.UUID;
Expand All @@ -40,6 +41,7 @@
import org.apache.cassandra.spark.bulkwriter.blobupload.BlobStreamResult;
import org.apache.cassandra.spark.bulkwriter.token.ReplicaAwareFailureHandler;
import org.apache.cassandra.spark.common.client.ClientException;
import org.apache.cassandra.spark.common.stats.JobStatsListener;
import org.apache.cassandra.spark.transports.storage.extensions.StorageTransportConfiguration;
import org.apache.cassandra.spark.transports.storage.extensions.StorageTransportExtension;
import org.apache.cassandra.spark.transports.storage.extensions.StorageTransportHandler;
Expand All @@ -65,6 +67,7 @@ public class CassandraBulkSourceRelation extends BaseRelation implements Inserta
private final SQLContext sqlContext;
private final JavaSparkContext sparkContext;
private final Broadcast<BulkWriterContext> broadcastContext;
private final JobStatsListener jobStatsListener;
private final BulkWriteValidator writeValidator;
private HeartbeatReporter heartbeatReporter;
private long startTimeNanos;
Expand All @@ -79,6 +82,15 @@ public CassandraBulkSourceRelation(BulkWriterContext writerContext, SQLContext s
ReplicaAwareFailureHandler<RingInstance> failureHandler = new ReplicaAwareFailureHandler<>(writerContext.cluster().getPartitioner());
this.writeValidator = new BulkWriteValidator(writerContext, failureHandler);
onCloudStorageTransport(ignored -> this.heartbeatReporter = new HeartbeatReporter());
this.jobStatsListener = new JobStatsListener((jobEventDetail) -> {
// Note: Consumers are called for all jobs and tasks. We only publish for the existing job
if (writerContext.job().getId().equals(jobEventDetail.internalJobID()))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also check for !internalJobId.isEmpty()?

{
writerContext.jobStats().publish(jobEventDetail.jobStats());
}
});

this.sparkContext.sc().addSparkListener(jobStatsListener);
}

@Override
Expand Down Expand Up @@ -114,7 +126,6 @@ public void insert(@NotNull Dataset<Row> data, boolean overwrite)
{
validateJob(overwrite);
this.startTimeNanos = System.nanoTime();

maybeEnableTransportExtension();
Tokenizer tokenizer = new Tokenizer(writerContext);
TableSchema tableSchema = writerContext.schema().getTableSchema();
Expand Down Expand Up @@ -186,7 +197,8 @@ private void persist(@NotNull JavaPairRDD<DecoratedKey, Object[]> sortedRDD, Str

long rowCount = streamResults.stream().mapToLong(res -> res.rowCount).sum();
long totalBytesWritten = streamResults.stream().mapToLong(res -> res.bytesWritten).sum();
boolean hasClusterTopologyChanged = writeResults.stream().anyMatch(WriteResult::isClusterResizeDetected);
boolean hasClusterTopologyChanged = writeResults.stream()
.anyMatch(WriteResult::isClusterResizeDetected);

onCloudStorageTransport(context -> {
LOGGER.info("Waiting for Cassandra to complete import slices. rows={} bytes={} cluster_resized={}",
Expand Down Expand Up @@ -217,16 +229,15 @@ private void persist(@NotNull JavaPairRDD<DecoratedKey, Object[]> sortedRDD, Str
markRestoreJobAsSucceeded(context);
});

LOGGER.info("Bulk writer job complete. rows={} bytes={} cluster_resized={}",
LOGGER.info("Bulk writer job complete. rows={} bytes={} cluster_resize={}",
rowCount,
totalBytesWritten,
hasClusterTopologyChanged);
publishSuccessfulJobStats(rowCount, totalBytesWritten, hasClusterTopologyChanged);
}
catch (Throwable throwable)
{
publishFailureJobStats(throwable.getMessage());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure what happens here, do we no longer publish failure stats?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we are not explicitly publishing failure stats at the point of failure. Instead, we rely on the job failure event, and the listener now publishes these stats.

LOGGER.error("Bulk Write Failed.", throwable);
LOGGER.error("Bulk Write Failed", throwable);
RuntimeException failure = new RuntimeException("Bulk Write to Cassandra has failed", throwable);
try
{
Expand Down Expand Up @@ -258,28 +269,17 @@ private void persist(@NotNull JavaPairRDD<DecoratedKey, Object[]> sortedRDD, Str

private void publishSuccessfulJobStats(long rowCount, long totalBytesWritten, boolean hasClusterTopologyChanged)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does it make sense to keep the Successful name on the method if we are ignoring failure stats?

{
writerContext.jobStats().publish(new HashMap<String, String>() // type declaration required to compile with java8
{{
Map<String, String> stats = new HashMap<String, String>()
{
{
put("jobId", writerContext.job().getId().toString());
put("transportInfo", writerContext.job().transportInfo().toString());
put("rowsWritten", Long.toString(rowCount));
put("bytesWritten", Long.toString(totalBytesWritten));
put("jobStatus", "Succeeded");
put("clusterResizeDetected", String.valueOf(hasClusterTopologyChanged));
put("jobElapsedTimeMillis", Long.toString(elapsedTimeMillis()));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why are we removing the jobElapsedTimeMillis stat?

}});
}

private void publishFailureJobStats(String reason)
{
writerContext.jobStats().publish(new HashMap<String, String>() // type declaration required to compile with java8
{{
put("jobId", writerContext.job().getId().toString());
put("transportInfo", writerContext.job().transportInfo().toString());
put("jobStatus", "Failed");
put("failureReason", reason);
put("jobElapsedTimeMillis", Long.toString(elapsedTimeMillis()));
}});
}
};
writerContext.jobStats().publish(stats);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,6 @@ protected CassandraBulkWriterContext(@NotNull BulkSparkConf conf,
{
this.conf = conf;
this.clusterInfo = clusterInfo;
clusterInfo.startupValidate();
this.jobStatsPublisher = new LogStatsPublisher();
lowestCassandraVersion = clusterInfo.getLowestCassandraVersion();
this.bridge = CassandraBridgeFactory.get(lowestCassandraVersion);
Expand Down Expand Up @@ -124,6 +123,7 @@ public static BulkWriterContext fromOptions(@NotNull SparkContext sparkContext,

BulkSparkConf conf = new BulkSparkConf(sparkContext.getConf(), strOptions);
CassandraClusterInfo clusterInfo = new CassandraClusterInfo(conf);
clusterInfo.startupValidate();
CassandraBulkWriterContext bulkWriterContext = new CassandraBulkWriterContext(conf, clusterInfo, dfSchema, sparkContext);
ShutdownHookManager.addShutdownHook(org.apache.spark.util.ShutdownHookManager.TEMP_DIR_SHUTDOWN_PRIORITY(),
ScalaFunctions.wrapLambda(bulkWriterContext::shutdown));
Expand All @@ -135,12 +135,11 @@ private void publishInitialJobStats(String sparkVersion)
{
Map<String, String> initialJobStats = new HashMap<String, String>() // type declaration required to compile with java8
{{
put("jobId", jobInfo.getId().toString());
put("sparkVersion", sparkVersion);
put("keyspace", jobInfo.getId().toString());
put("table", jobInfo.getId().toString());
put("keyspace", jobInfo.qualifiedTableName().keyspace());
put("table", jobInfo.qualifiedTableName().table());
}};
jobStatsPublisher.publish(initialJobStats);
publish(initialJobStats);
}

@Override
Expand Down Expand Up @@ -256,4 +255,9 @@ protected TableSchema initializeTableSchema(@NotNull BulkSparkConf conf,
lowestCassandraVersion,
jobInfo.qualifiedTableName().quoteIdentifiers());
}

public void publish(Map<String, String> stats)
{
LOGGER.info("Job Stats:" + stats);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,171 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.cassandra.spark.common.stats;

import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.stream.Collectors;

import org.apache.commons.math3.stat.StatUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.cassandra.spark.JobEventDetail;
import org.apache.spark.executor.TaskMetrics;
import org.apache.spark.scheduler.JobFailed;
import org.apache.spark.scheduler.SparkListener;
import org.apache.spark.scheduler.SparkListenerJobEnd;
import org.apache.spark.scheduler.SparkListenerJobStart;
import org.apache.spark.scheduler.SparkListenerTaskEnd;

/**
* Spark listener implementation to capture stats on completion of jobs and tasks.
*/
public class JobStatsListener extends SparkListener
{
private final Map<Integer, Integer> jobIdToTaskRetryStats = new HashMap<>();
private final Map<Integer, Set<TaskMetrics>> jobIdToTaskMetrics = new HashMap<>();
private final Map<Integer, Long> jobIdToStartTimes = new HashMap<>();
private final Map<Integer, String> internalJobIdMapping = new HashMap<>();

private static int jobId = -1;
private static final Logger LOGGER = LoggerFactory.getLogger(JobStatsListener.class);
private final Consumer<JobEventDetail> jobCompletionConsumer;

public JobStatsListener(Consumer<JobEventDetail> jobCompletionConsumer)
{
this.jobCompletionConsumer = jobCompletionConsumer;
}

@Override
public synchronized void onTaskEnd(SparkListenerTaskEnd taskEnd)
{
try
{
// Calculate max task retries across all tasks in job
int attempt = taskEnd.taskInfo().attemptNumber();
jobIdToTaskRetryStats.compute(jobId, (k, v) -> (v == null || attempt > v) ? attempt : v);
// Persist all task metrics for the job - across all stages
jobIdToTaskMetrics.computeIfAbsent(jobId, k -> new HashSet<>()).add(taskEnd.taskMetrics());
LOGGER.debug("Task END for jobId:{} task:{} task attempt:{}} Reason:{}",
jobId,
taskEnd.taskInfo().taskId(),
taskEnd.taskInfo().attemptNumber(),
taskEnd.reason());

}
catch (Exception e)
{
LOGGER.warn("Failed to process job stats for the task completion event with jobId: {}",
internalJobIdMapping.get(jobId), e);
}
}

@Override
public void onJobStart(SparkListenerJobStart jobStart)
{
String internalJobId = (String) jobStart.properties().get("spark.jobGroup.id");
try
{

jobId = Integer.valueOf(jobStart.jobId());
internalJobIdMapping.put(jobId, internalJobId);
jobIdToStartTimes.put(jobId, System.nanoTime());
}
catch (Exception e)
{
LOGGER.warn("Failed to process job stats for the job start event with jobId: {}", internalJobId, e);
}
}

@Override
public void onJobEnd(SparkListenerJobEnd jobEnd)
{
try
{
boolean jobFailed = false;
String reason = "null";
if (jobEnd.jobResult() instanceof JobFailed)
{
jobFailed = true;
JobFailed result = (JobFailed) jobEnd.jobResult();
reason = result.exception().getCause().getMessage();
}

long elapsedTimeMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - jobIdToStartTimes.get(jobId));
String internalJobId = internalJobIdMapping.get(jobId);
String jobStatus = (jobFailed) ? "Failed" : "Succeeded";
Map<String, String> jobStats = new HashMap<>();
jobStats.put("jobId", internalJobId);
jobStats.put("jobStatus", jobStatus);
jobStats.put("failureReason", reason);
jobStats.put("jobElapsedTimeMillis", String.valueOf(elapsedTimeMillis));

LOGGER.debug("Job END for jobId:{} status:{} Reason:{} ElapsedTime: {}",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is an extra space after ElapsedTime

jobId,
jobStatus,
reason,
elapsedTimeMillis);

jobStats.putAll(getJobMetrics(jobId));
jobCompletionConsumer.accept(new JobEventDetail(internalJobId, jobStats));
cleanup(jobId);
}
catch (Exception e)
{
LOGGER.warn("Failed to process job stats for the job completion event with jobId: {}",
internalJobIdMapping.get(jobId), e);
}

}

public synchronized Map<String, String> getJobMetrics(int jobId)
{
Map<String, String> jobMetrics = new HashMap<>();
if (jobIdToTaskMetrics.containsKey(jobId))
{
List<Long> runTimes = jobIdToTaskMetrics.get(jobId)
.stream()
.map(TaskMetrics::executorRunTime)
.collect(Collectors.toList());

double[] runTimesArray = runTimes.stream().mapToDouble(Long::doubleValue).toArray();
jobMetrics.put("maxTaskRuntimeMillis", String.valueOf(StatUtils.max(runTimesArray)));
jobMetrics.put("meanTaskRuntimeMillis", String.valueOf(StatUtils.mean(runTimesArray)));
jobMetrics.put("p50TaskRuntimeMillis", String.valueOf(StatUtils.percentile(runTimesArray, 50)));
jobMetrics.put("p95TaskRuntimeMillis", String.valueOf(StatUtils.percentile(runTimesArray, 95)));
jobMetrics.put("maxTaskRetriesMillis", String.valueOf(jobIdToTaskRetryStats.get(jobId)));
}
return jobMetrics;
}

private void cleanup(int jobId)
{
jobIdToStartTimes.remove(jobId);
internalJobIdMapping.remove(jobId);
jobIdToTaskMetrics.remove(jobId);
jobIdToTaskRetryStats.remove(jobId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,7 @@ public interface JobStatsPublisher
{
/**
* Publish the job attributes to be persisted and summarized
*
* @param stats the stats to publish
* @param stats mapping of the metric names and their values
*/
void publish(Map<String, String> stats);
}
Loading