From ee6a8d7eaaafafb7eee6f9f2d633759b12732062 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 9 May 2016 11:13:29 -0700 Subject: [MINOR][SQL] Enhance the exception message if checkpointLocation is not set Enhance the exception message when `checkpointLocation` is not set, previously the message is: ``` java.util.NoSuchElementException: None.get at scala.None$.get(Option.scala:347) at scala.None$.get(Option.scala:345) at org.apache.spark.sql.DataFrameWriter$$anonfun$8.apply(DataFrameWriter.scala:338) at org.apache.spark.sql.DataFrameWriter$$anonfun$8.apply(DataFrameWriter.scala:338) at scala.collection.MapLike$class.getOrElse(MapLike.scala:128) at scala.collection.AbstractMap.getOrElse(Map.scala:59) at org.apache.spark.sql.DataFrameWriter.startStream(DataFrameWriter.scala:337) at org.apache.spark.sql.DataFrameWriter.startStream(DataFrameWriter.scala:277) ... 48 elided ``` This is not so meaningful, so changing to make it more specific. Local verified. Author: jerryshao Closes #12998 from jerryshao/improve-exception-message. --- .../main/scala/org/apache/spark/sql/DataFrameWriter.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 0793b62fae..6b1ccbec61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -334,9 +334,15 @@ final class DataFrameWriter private[sql](df: DataFrame) { partitionColumns = normalizedParCols.getOrElse(Nil)) val queryName = extraOptions.getOrElse("queryName", StreamExecution.nextName) - val checkpointLocation = extraOptions.getOrElse("checkpointLocation", - new Path(df.sparkSession.sessionState.conf.checkpointLocation.get, queryName).toUri.toString - ) + val checkpointLocation = extraOptions.get("checkpointLocation") + .orElse { + df.sparkSession.sessionState.conf.checkpointLocation.map { l => + new Path(l, queryName).toUri.toString + } + }.getOrElse { + throw new AnalysisException("checkpointLocation must be specified either " + + "through option() or SQLConf") + } df.sparkSession.sessionState.continuousQueryManager.startQuery( queryName, -- cgit v1.2.3