@@ -32,6 +32,8 @@ import org.apache.spark.sql.execution.debug._
3232import org .apache .spark .sql .internal .SQLConf
3333import org .apache .spark .sql .types ._
3434
35+ import scala .collection .immutable .ListMap
36+
3537/**
3638 * A logical command that is executed for its side-effects. `RunnableCommand`s are
3739 * wrapped in `ExecutedCommand` during execution.
@@ -181,17 +183,18 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm
181183 // Queries all key-value pairs that are set in the SQLConf of the sqlContext.
182184 case None =>
183185 val runFunc = (sqlContext : SQLContext ) => {
184- sqlContext.getAllConfs.map { case (k, v) => Row (k, v) }.toSeq
186+ sqlContext.getAllConfs.toSeq.sortBy(_._1).map { case (k, v) => Row (k, v) } ++
187+ getEnvList(withDoc = false )
185188 }
186189 (keyValueOutput, runFunc)
187190
188191 // Queries all properties along with their default values and docs that are defined in the
189192 // SQLConf of the sqlContext.
190193 case Some ((" -v" , None )) =>
191194 val runFunc = (sqlContext : SQLContext ) => {
192- sqlContext.conf.getAllDefinedConfs.map { case (key, defaultValue, doc) =>
195+ sqlContext.conf.getAllDefinedConfs.sortBy(_._1). map { case (key, defaultValue, doc) =>
193196 Row (key, defaultValue, doc)
194- }
197+ } ++ getEnvList(withDoc = true )
195198 }
196199 val schema = StructType (
197200 StructField (" key" , StringType , false ) ::
@@ -225,6 +228,20 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm
225228
226229 override def run (sqlContext : SQLContext ): Seq [Row ] = runFunc(sqlContext)
227230
231+ /**
232+ * get the system environment properties as a sequence
233+ * @param withDoc whether the result has a doc column or not
234+ * @return the sequence of the rows containing the key/value pair of system properties
235+ */
236+ private def getEnvList (withDoc : Boolean ) = {
237+ import scala .collection .JavaConverters ._
238+ System .getenv().asScala.toSeq.sortBy(_._1).map {
239+ case (k, v) => if (withDoc) Row (s " env: $k" , v, " " ) else Row (s " env: $k" , v)
240+ } ++
241+ System .getProperties.asScala.toSeq.sortBy(_._1).map {
242+ case (k, v) => if (withDoc) Row (s " system: $k" , v, " " ) else Row (s " system: $k" , v)
243+ }
244+ }
228245}
229246
230247/**
0 commit comments