-
Notifications
You must be signed in to change notification settings - Fork 232
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Qualification tool: Add filter based on appName (#2934)
* Additional filtering features for qualification tool * refactor, read event logs multithreaded * addressed review comments and added appName filter * Add tests and update filter appName feature * addressed review comments Signed-off-by: Niranjan Artal <nartal@nvidia.com>
- Loading branch information
Showing
10 changed files
with
214 additions
and
9 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
102 changes: 102 additions & 0 deletions
102
tools/src/main/scala/org/apache/spark/sql/rapids/tool/AppFilterImpl.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,102 @@ | ||
/* | ||
* Copyright (c) 2021, NVIDIA CORPORATION. | ||
* | ||
* Licensed 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.spark.sql.rapids.tool | ||
|
||
import java.util.concurrent.{ConcurrentLinkedQueue, Executors, ThreadPoolExecutor, TimeUnit} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import com.google.common.util.concurrent.ThreadFactoryBuilder | ||
import com.nvidia.spark.rapids.tool.EventLogInfo | ||
import com.nvidia.spark.rapids.tool.qualification.QualificationArgs | ||
import org.apache.hadoop.conf.Configuration | ||
|
||
import org.apache.spark.internal.Logging | ||
|
||
class AppFilterImpl( | ||
numRows: Int, | ||
hadoopConf: Configuration, | ||
timeout: Option[Long], | ||
nThreads: Int) extends Logging { | ||
|
||
private val appsForFiltering = new ConcurrentLinkedQueue[AppFilterReturnParameters]() | ||
// default is 24 hours | ||
private val waitTimeInSec = timeout.getOrElse(60 * 60 * 24L) | ||
|
||
private val threadFactory = new ThreadFactoryBuilder() | ||
.setDaemon(true).setNameFormat("qualAppFilter" + "-%d").build() | ||
logInfo(s"Threadpool size is $nThreads") | ||
private val qualFilterthreadPool = Executors.newFixedThreadPool(nThreads, threadFactory) | ||
.asInstanceOf[ThreadPoolExecutor] | ||
|
||
private class FilterThread(path: EventLogInfo) extends Runnable { | ||
def run: Unit = filterEventLog(path, numRows, hadoopConf) | ||
} | ||
|
||
def filterEventLogs( | ||
allPaths: Seq[EventLogInfo], | ||
appArgs: QualificationArgs): Seq[EventLogInfo] = { | ||
allPaths.foreach { path => | ||
try { | ||
qualFilterthreadPool.submit(new FilterThread(path)) | ||
} catch { | ||
case e: Exception => | ||
logError(s"Unexpected exception submitting log ${path.eventLog.toString}, skipping!", e) | ||
} | ||
} | ||
// wait for the threads to finish processing the files | ||
qualFilterthreadPool.shutdown() | ||
if (!qualFilterthreadPool.awaitTermination(waitTimeInSec, TimeUnit.SECONDS)) { | ||
logError(s"Processing log files took longer then $waitTimeInSec seconds," + | ||
" stopping processing any more event logs") | ||
qualFilterthreadPool.shutdownNow() | ||
} | ||
|
||
// This will be required to do the actual filtering | ||
val apps = appsForFiltering.asScala | ||
|
||
val filterAppName = appArgs.applicationName.getOrElse("") | ||
if (appArgs.applicationName.isSupplied && filterAppName.nonEmpty) { | ||
val filtered = apps.filter { app => | ||
val appNameOpt = app.appInfo.map(_.appName) | ||
if (appNameOpt.isDefined) { | ||
appNameOpt.get.equals(filterAppName) | ||
} else { | ||
// in complete log file | ||
false | ||
} | ||
} | ||
filtered.map(_.eventlog).toSeq | ||
} else { | ||
apps.map(x => x.eventlog).toSeq | ||
} | ||
} | ||
|
||
case class AppFilterReturnParameters( | ||
appInfo: Option[ApplicationStartInfo], | ||
eventlog: EventLogInfo) | ||
|
||
private def filterEventLog( | ||
path: EventLogInfo, | ||
numRows: Int, | ||
hadoopConf: Configuration): Unit = { | ||
|
||
val startAppInfo = new FilterAppInfo(numRows, path, hadoopConf) | ||
val appInfo = AppFilterReturnParameters(startAppInfo.appInfo, path) | ||
appsForFiltering.add(appInfo) | ||
} | ||
} |
55 changes: 55 additions & 0 deletions
55
tools/src/main/scala/org/apache/spark/sql/rapids/tool/FilterAppInfo.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
/* | ||
* Copyright (c) 2021, NVIDIA CORPORATION. | ||
* | ||
* Licensed 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.spark.sql.rapids.tool | ||
|
||
import com.nvidia.spark.rapids.tool.EventLogInfo | ||
import org.apache.hadoop.conf.Configuration | ||
|
||
import org.apache.spark.scheduler.{SparkListenerApplicationStart, SparkListenerEvent} | ||
|
||
case class ApplicationStartInfo( | ||
appName: String, | ||
startTime: Long) | ||
|
||
class FilterAppInfo( | ||
numOutputRows: Int, | ||
eventLogInfo: EventLogInfo, | ||
hadoopConf: Configuration) extends AppBase(numOutputRows, eventLogInfo, hadoopConf) { | ||
|
||
def doSparkListenerApplicationStart( | ||
event: SparkListenerApplicationStart): Unit = { | ||
logDebug("Processing event: " + event.getClass) | ||
val thisAppInfo = ApplicationStartInfo( | ||
event.appName, | ||
event.time | ||
) | ||
appInfo = Some(thisAppInfo) | ||
} | ||
|
||
var appInfo: Option[ApplicationStartInfo] = None | ||
|
||
override def processEvent(event: SparkListenerEvent): Boolean = { | ||
if (event.isInstanceOf[SparkListenerApplicationStart]) { | ||
doSparkListenerApplicationStart(event.asInstanceOf[SparkListenerApplicationStart]) | ||
true | ||
} else { | ||
false | ||
} | ||
} | ||
|
||
processEvents() | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters