Back to home page

OSCL-LXR

 
 

    


0001 /*
0002  * Licensed to the Apache Software Foundation (ASF) under one or more
0003  * contributor license agreements.  See the NOTICE file distributed with
0004  * this work for additional information regarding copyright ownership.
0005  * The ASF licenses this file to You under the Apache License, Version 2.0
0006  * (the "License"); you may not use this file except in compliance with
0007  * the License.  You may obtain a copy of the License at
0008  *
0009  *    http://www.apache.org/licenses/LICENSE-2.0
0010  *
0011  * Unless required by applicable law or agreed to in writing, software
0012  * distributed under the License is distributed on an "AS IS" BASIS,
0013  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0014  * See the License for the specific language governing permissions and
0015  * limitations under the License.
0016  */
0017 
0018 package org.apache.spark;
0019 
0020 import org.apache.spark.scheduler.*;
0021 
0022 /**
0023  * Class that allows users to receive all SparkListener events.
0024  * Users should override the onEvent method.
0025  *
0026  * This is a concrete Java class in order to ensure that we don't forget to update it when adding
0027  * new methods to SparkListener: forgetting to add a method will result in a compilation error (if
0028  * this was a concrete Scala class, default implementations of new event handlers would be inherited
0029  * from the SparkListener trait).
0030  */
0031 public class SparkFirehoseListener implements SparkListenerInterface {
0032 
0033   public void onEvent(SparkListenerEvent event) { }
0034 
0035   @Override
0036   public final void onStageCompleted(SparkListenerStageCompleted stageCompleted) {
0037     onEvent(stageCompleted);
0038   }
0039 
0040   @Override
0041   public final void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) {
0042     onEvent(stageSubmitted);
0043   }
0044 
0045   @Override
0046   public final void onTaskStart(SparkListenerTaskStart taskStart) {
0047     onEvent(taskStart);
0048   }
0049 
0050   @Override
0051   public final void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) {
0052     onEvent(taskGettingResult);
0053   }
0054 
0055   @Override
0056   public final void onTaskEnd(SparkListenerTaskEnd taskEnd) {
0057     onEvent(taskEnd);
0058   }
0059 
0060   @Override
0061   public final void onJobStart(SparkListenerJobStart jobStart) {
0062     onEvent(jobStart);
0063   }
0064 
0065   @Override
0066   public final void onJobEnd(SparkListenerJobEnd jobEnd) {
0067     onEvent(jobEnd);
0068   }
0069 
0070   @Override
0071   public final void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) {
0072     onEvent(environmentUpdate);
0073   }
0074 
0075   @Override
0076   public final void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) {
0077     onEvent(blockManagerAdded);
0078   }
0079 
0080   @Override
0081   public final void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) {
0082     onEvent(blockManagerRemoved);
0083   }
0084 
0085   @Override
0086   public final void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) {
0087     onEvent(unpersistRDD);
0088   }
0089 
0090   @Override
0091   public final void onApplicationStart(SparkListenerApplicationStart applicationStart) {
0092     onEvent(applicationStart);
0093   }
0094 
0095   @Override
0096   public final void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) {
0097     onEvent(applicationEnd);
0098   }
0099 
0100   @Override
0101   public final void onExecutorMetricsUpdate(
0102       SparkListenerExecutorMetricsUpdate executorMetricsUpdate) {
0103     onEvent(executorMetricsUpdate);
0104   }
0105 
0106   @Override
0107   public final void onStageExecutorMetrics(
0108       SparkListenerStageExecutorMetrics executorMetrics) {
0109     onEvent(executorMetrics);
0110   }
0111 
0112   @Override
0113   public final void onExecutorAdded(SparkListenerExecutorAdded executorAdded) {
0114     onEvent(executorAdded);
0115   }
0116 
0117   @Override
0118   public final void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) {
0119     onEvent(executorRemoved);
0120   }
0121 
0122   @Override
0123   public final void onExecutorBlacklisted(SparkListenerExecutorBlacklisted executorBlacklisted) {
0124     onEvent(executorBlacklisted);
0125   }
0126 
0127   @Override
0128   public void onExecutorBlacklistedForStage(
0129       SparkListenerExecutorBlacklistedForStage executorBlacklistedForStage) {
0130     onEvent(executorBlacklistedForStage);
0131   }
0132 
0133   @Override
0134   public void onNodeBlacklistedForStage(
0135       SparkListenerNodeBlacklistedForStage nodeBlacklistedForStage) {
0136     onEvent(nodeBlacklistedForStage);
0137   }
0138 
0139   @Override
0140   public final void onExecutorUnblacklisted(
0141       SparkListenerExecutorUnblacklisted executorUnblacklisted) {
0142     onEvent(executorUnblacklisted);
0143   }
0144 
0145   @Override
0146   public final void onNodeBlacklisted(SparkListenerNodeBlacklisted nodeBlacklisted) {
0147     onEvent(nodeBlacklisted);
0148   }
0149 
0150   @Override
0151   public final void onNodeUnblacklisted(SparkListenerNodeUnblacklisted nodeUnblacklisted) {
0152     onEvent(nodeUnblacklisted);
0153   }
0154 
0155   @Override
0156   public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) {
0157     onEvent(blockUpdated);
0158   }
0159 
0160   @Override
0161   public void onSpeculativeTaskSubmitted(SparkListenerSpeculativeTaskSubmitted speculativeTask) {
0162     onEvent(speculativeTask);
0163   }
0164 
0165   @Override
0166   public void onOtherEvent(SparkListenerEvent event) {
0167     onEvent(event);
0168   }
0169 }