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

Improve flint error handling #335

Merged
merged 3 commits into from
May 11, 2024
Merged
Show file tree
Hide file tree
Changes from 2 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
2 changes: 2 additions & 0 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -208,6 +208,8 @@ lazy val sparkSqlApplication = (project in file("spark-sql-application"))
libraryDependencies ++= deps(sparkVersion),
libraryDependencies ++= Seq(
"com.typesafe.play" %% "play-json" % "2.9.2",
"com.amazonaws" % "aws-java-sdk-glue" % "1.12.568" % "provided"
exclude ("com.fasterxml.jackson.core", "jackson-databind"),
// handle AmazonS3Exception
"com.amazonaws" % "aws-java-sdk-s3" % "1.12.568" % "provided"
// the transitive jackson.core dependency conflicts with existing scala
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,11 @@ public final class MetricConstants {
*/
public static final String S3_ERR_CNT_METRIC = "s3.error.count";

/**
* Metric name for counting the errors encountered with Amazon Glue operations.
*/
public static final String GLUE_ERR_CNT_METRIC = "glue.error.count";

/**
* Metric name for counting the number of sessions currently running.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,10 @@ package org.apache.spark.sql

import java.util.Locale

import com.amazonaws.services.glue.model.{AccessDeniedException, AWSGlueException}
import com.amazonaws.services.s3.model.AmazonS3Exception
import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.module.scala.DefaultScalaModule
import org.apache.commons.text.StringEscapeUtils.unescapeJava
import org.opensearch.flint.core.IRestHighLevelClient
import org.opensearch.flint.core.metrics.MetricConstants
Expand All @@ -17,13 +20,17 @@ import play.api.libs.json._
import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.flint.config.FlintSparkConf
import org.apache.spark.sql.flint.config.FlintSparkConf.REFRESH_POLICY
import org.apache.spark.sql.types._
import org.apache.spark.sql.util._

trait FlintJobExecutor {
this: Logging =>

val mapper = new ObjectMapper()
mapper.registerModule(DefaultScalaModule)

var currentTimeProvider: TimeProvider = new RealTimeProvider()
var threadPoolFactory: ThreadPoolFactory = new DefaultThreadPoolFactory()
var envinromentProvider: EnvironmentProvider = new RealEnvironment()
Expand Down Expand Up @@ -65,6 +72,9 @@ trait FlintJobExecutor {
"sessionId": {
"type": "keyword"
},
"jobType": {
"type": "keyword"
},
"updateTime": {
"type": "date",
"format": "strict_date_time||epoch_millis"
Expand Down Expand Up @@ -190,6 +200,7 @@ trait FlintJobExecutor {
StructField("queryId", StringType, nullable = true),
StructField("queryText", StringType, nullable = true),
StructField("sessionId", StringType, nullable = true),
StructField("jobType", StringType, nullable = true),
// number is not nullable
StructField("updateTime", LongType, nullable = false),
StructField("queryRunTime", LongType, nullable = true)))
Expand Down Expand Up @@ -218,6 +229,7 @@ trait FlintJobExecutor {
queryId,
query,
sessionId,
spark.conf.get(FlintSparkConf.JOB_TYPE.key),
endTime,
endTime - startTime))

Expand Down Expand Up @@ -248,6 +260,7 @@ trait FlintJobExecutor {
StructField("queryId", StringType, nullable = true),
StructField("queryText", StringType, nullable = true),
StructField("sessionId", StringType, nullable = true),
StructField("jobType", StringType, nullable = true),
// number is not nullable
StructField("updateTime", LongType, nullable = false),
StructField("queryRunTime", LongType, nullable = true)))
Expand All @@ -267,6 +280,7 @@ trait FlintJobExecutor {
queryId,
query,
sessionId,
spark.conf.get(FlintSparkConf.JOB_TYPE.key),
endTime,
endTime - startTime))

Expand Down Expand Up @@ -411,68 +425,58 @@ trait FlintJobExecutor {
private def handleQueryException(
e: Exception,
message: String,
spark: SparkSession,
dataSource: String,
query: String,
queryId: String,
sessionId: String): String = {
val error = s"$message: ${e.getMessage}"
logError(error, e)
error
errorSource: Option[String] = None,
statusCode: Option[Int] = None): String = {

val errorDetails = Map("Message" -> s"$message: ${e.getMessage}") ++
errorSource.map("ErrorSource" -> _) ++
statusCode.map(code => "StatusCode" -> code.toString)

val errorJson = mapper.writeValueAsString(errorDetails)
logError(errorJson, e)
errorJson
}

def getRootCause(e: Throwable): Throwable = {
if (e.getCause == null) e
else getRootCause(e.getCause)
}

def processQueryException(
ex: Exception,
spark: SparkSession,
dataSource: String,
query: String,
queryId: String,
sessionId: String): String = {
/**
* This method converts query exception into error string, which then persist to query result
* metadata
*/
def processQueryException(ex: Exception): String = {
getRootCause(ex) match {
case r: ParseException =>
handleQueryException(r, "Syntax error", spark, dataSource, query, queryId, sessionId)
handleQueryException(r, "Syntax error")
case r: AmazonS3Exception =>
incrementCounter(MetricConstants.S3_ERR_CNT_METRIC)
handleQueryException(
r,
"Fail to read data from S3. Cause",
spark,
dataSource,
query,
queryId,
sessionId)
case r: AnalysisException =>
Some(r.getServiceName),
Some(r.getStatusCode))
case r: AWSGlueException =>

Choose a reason for hiding this comment

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

Can we also add handling for permission errors to S3, OpenSearch as well?

incrementCounter(MetricConstants.GLUE_ERR_CNT_METRIC)
// Redact Access denied in AWS Glue service
r match {
case accessDenied: AccessDeniedException =>
accessDenied.setErrorMessage(
"Access denied in AWS Glue service. Please check permissions.")
case _ => // No additional action for other types of AWSGlueException
}
handleQueryException(
r,
"Fail to analyze query. Cause",
spark,
dataSource,
query,
queryId,
sessionId)
"Fail to read data from Glue. Cause",
Some(r.getServiceName),
Some(r.getStatusCode))
case r: AnalysisException =>
handleQueryException(r, "Fail to analyze query. Cause")
case r: SparkException =>
handleQueryException(
r,
"Spark exception. Cause",
spark,
dataSource,
query,
queryId,
sessionId)
handleQueryException(r, "Spark exception. Cause")
case r: Exception =>
handleQueryException(
r,
"Fail to run query, cause",
spark,
dataSource,
query,
queryId,
sessionId)
handleQueryException(r, "Fail to run query. Cause")
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -545,19 +545,8 @@ object FlintREPL extends Logging with FlintJobExecutor {
currentTimeProvider)
}

def processQueryException(
ex: Exception,
spark: SparkSession,
dataSource: String,
flintCommand: FlintCommand,
sessionId: String): String = {
val error = super.processQueryException(
ex,
spark,
dataSource,
flintCommand.query,
flintCommand.queryId,
sessionId)
def processQueryException(ex: Exception, flintCommand: FlintCommand): String = {
val error = super.processQueryException(ex)
flintCommand.fail()
flintCommand.error = Some(error)
error
Expand Down Expand Up @@ -724,7 +713,7 @@ object FlintREPL extends Logging with FlintJobExecutor {
sessionId,
startTime)
case e: Exception =>
val error = processQueryException(e, spark, dataSource, flintCommand.query, "", "")
val error = processQueryException(e, flintCommand)
Some(
handleCommandFailureAndGetFailedData(
spark,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ case class JobOperator(
dataToWrite = Some(
getFailedData(spark, dataSource, error, "", query, "", startTime, currentTimeProvider))
case e: Exception =>
val error = processQueryException(e, spark, dataSource, query, "", "")
val error = processQueryException(e)
dataToWrite = Some(
getFailedData(spark, dataSource, error, "", query, "", startTime, currentTimeProvider))
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,14 +6,15 @@
package org.apache.spark.sql

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.flint.config.FlintSparkConf
import org.apache.spark.sql.types._
import org.apache.spark.sql.util.{CleanerFactory, MockTimeProvider}

class FlintJobTest extends SparkFunSuite with JobMatchers {

val spark =
SparkSession.builder().appName("Test").master("local").getOrCreate()

spark.conf.set(FlintSparkConf.JOB_TYPE.key, "streaming")
// Define input dataframe
val inputSchema = StructType(
Seq(
Expand All @@ -38,6 +39,7 @@ class FlintJobTest extends SparkFunSuite with JobMatchers {
StructField("queryId", StringType, nullable = true),
StructField("queryText", StringType, nullable = true),
StructField("sessionId", StringType, nullable = true),
StructField("jobType", StringType, nullable = true),
StructField("updateTime", LongType, nullable = false),
StructField("queryRunTime", LongType, nullable = false)))

Expand All @@ -61,6 +63,7 @@ class FlintJobTest extends SparkFunSuite with JobMatchers {
"10",
"select 1",
"20",
"streaming",
currentTime,
queryRunTime))
val expected: DataFrame =
Expand Down Expand Up @@ -93,7 +96,7 @@ class FlintJobTest extends SparkFunSuite with JobMatchers {
"""{"dynamic":false,"properties":{"result":{"type":"object"},"schema":{"type":"object"},
|"jobRunId":{"type":"keyword"},"applicationId":{
|"type":"keyword"},"dataSourceName":{"type":"keyword"},"status":{"type":"keyword"}}}
|"error":{"type":"text"}}}
|"error":{"type":"text"}, "jobType":{"type":"keyword"}}}
|""".stripMargin
assert(FlintJob.isSuperset(input, mapping))
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import scala.concurrent.duration._
import scala.concurrent.duration.{Duration, MINUTES}
import scala.reflect.runtime.universe.TypeTag

import com.amazonaws.services.glue.model.AccessDeniedException
import com.codahale.metrics.Timer
import org.mockito.ArgumentMatchersSugar
import org.mockito.Mockito._
Expand Down Expand Up @@ -216,6 +217,7 @@ class FlintREPLTest
StructField("queryId", StringType, nullable = true),
StructField("queryText", StringType, nullable = true),
StructField("sessionId", StringType, nullable = true),
StructField("jobType", StringType, nullable = true),
StructField("updateTime", LongType, nullable = false),
StructField("queryRunTime", LongType, nullable = false)))

Expand All @@ -235,10 +237,11 @@ class FlintREPLTest
"10",
"select 1",
"20",
"interactive",
currentTime,
queryRunTime))
val spark = SparkSession.builder().appName("Test").master("local").getOrCreate()

spark.conf.set(FlintSparkConf.JOB_TYPE.key, FlintSparkConf.JOB_TYPE.defaultValue.get)
val expected =
spark.createDataFrame(spark.sparkContext.parallelize(expectedRows), expectedSchema)

Expand Down Expand Up @@ -436,6 +439,31 @@ class FlintREPLTest
assert(result)
}

test("processQueryException should handle exceptions, fail the command, and set the error") {
val exception = new AccessDeniedException(
"Unable to verify existence of default database: com.amazonaws.services.glue.model.AccessDeniedException: " +
"User: ****** is not authorized to perform: glue:GetDatabase on resource: ****** " +
"because no identity-based policy allows the glue:GetDatabase action")
exception.setStatusCode(400)
exception.setErrorCode("AccessDeniedException")
exception.setServiceName("AWSGlue")

val mockFlintCommand = mock[FlintCommand]
val expectedError = (
"""{"Message":"Fail to read data from Glue. Cause: Access denied in AWS Glue service. Please check permissions. (Service: AWSGlue; """ +
"""Status Code: 400; Error Code: AccessDeniedException; Request ID: null; Proxy: null)",""" +
""""ErrorSource":"AWSGlue","StatusCode":"400"}"""
)

val result = FlintREPL.processQueryException(exception, mockFlintCommand)

result shouldEqual expectedError
verify(mockFlintCommand).fail()
verify(mockFlintCommand).error = Some(expectedError)

assert(result == expectedError)
}

test("Doc Exists and excludeJobIds is an ArrayList Containing JobId") {
val sessionId = "session123"
val jobId = "jobABC"
Expand Down Expand Up @@ -547,10 +575,13 @@ class FlintREPLTest
test("executeAndHandle should handle TimeoutException properly") {
val mockSparkSession = mock[SparkSession]
val mockFlintCommand = mock[FlintCommand]
val mockConf = mock[RuntimeConfig]
when(mockSparkSession.conf).thenReturn(mockConf)
when(mockSparkSession.conf.get(FlintSparkConf.JOB_TYPE.key))
.thenReturn(FlintSparkConf.JOB_TYPE.defaultValue.get)
// val mockExecutionContextExecutor: ExecutionContextExecutor = mock[ExecutionContextExecutor]
val threadPool = ThreadUtils.newDaemonThreadPoolScheduledExecutor("flint-repl", 1)
implicit val executionContext = ExecutionContext.fromExecutor(threadPool)

try {
val dataSource = "someDataSource"
val sessionId = "someSessionId"
Expand Down Expand Up @@ -596,6 +627,10 @@ class FlintREPLTest

test("executeAndHandle should handle ParseException properly") {
val mockSparkSession = mock[SparkSession]
val mockConf = mock[RuntimeConfig]
when(mockSparkSession.conf).thenReturn(mockConf)
when(mockSparkSession.conf.get(FlintSparkConf.JOB_TYPE.key))
.thenReturn(FlintSparkConf.JOB_TYPE.defaultValue.get)
val flintCommand =
new FlintCommand(
"Running",
Expand All @@ -606,7 +641,6 @@ class FlintREPLTest
None)
val threadPool = ThreadUtils.newDaemonThreadPoolScheduledExecutor("flint-repl", 1)
implicit val executionContext = ExecutionContext.fromExecutor(threadPool)

try {
val dataSource = "someDataSource"
val sessionId = "someSessionId"
Expand Down Expand Up @@ -1020,6 +1054,11 @@ class FlintREPLTest
val sparkContext = mock[SparkContext]
when(mockSparkSession.sparkContext).thenReturn(sparkContext)

val mockConf = mock[RuntimeConfig]
when(mockSparkSession.conf).thenReturn(mockConf)
when(mockSparkSession.conf.get(FlintSparkConf.JOB_TYPE.key))
.thenReturn(FlintSparkConf.JOB_TYPE.defaultValue.get)

when(expectedDataFrame.toDF(any[Seq[String]]: _*)).thenReturn(expectedDataFrame)

val flintSessionIndexUpdater = mock[OpenSearchUpdater]
Expand Down
Loading