bbotella commented on code in PR #51:
URL:
https://github.com/apache/cassandra-analytics/pull/51#discussion_r1611749908
##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java:
##########
@@ -258,28 +269,17 @@ private void persist(@NotNull JavaPairRDD<DecoratedKey,
Object[]> sortedRDD, Str
private void publishSuccessfulJobStats(long rowCount, long
totalBytesWritten, boolean hasClusterTopologyChanged)
{
- 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()));
Review Comment:
Why are we removing the jobElapsedTimeMillis stat?
##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java:
##########
@@ -258,28 +269,17 @@ private void persist(@NotNull JavaPairRDD<DecoratedKey,
Object[]> sortedRDD, Str
private void publishSuccessfulJobStats(long rowCount, long
totalBytesWritten, boolean hasClusterTopologyChanged)
Review Comment:
Does it make sense to keep the Successful name on the method if we are
ignoring failure stats?
##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java:
##########
@@ -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()))
Review Comment:
Should we also check for `!internalJobId.isEmpty()`?
##########
cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java:
##########
@@ -81,12 +82,14 @@ public class MockBulkWriterContext implements
BulkWriterContext, ClusterInfo, Jo
new CqlField.CqlType[]{mockCqlType(INT), mockCqlType(DATE),
mockCqlType(VARCHAR), mockCqlType(INT)});
private ConsistencyLevel.CL consistencyLevel;
private int sstableDataSizeInMB = 128;
+ private int sstableWriteBatchSize = 2;
Review Comment:
What is this new variable doing? Where is it used?
##########
cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/stats/JobStatsListener.java:
##########
@@ -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: {}",
Review Comment:
There is an extra space after ElapsedTime
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]