diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 1631e2d..eb65ac1 100644 --- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -2040,6 +2040,7 @@ public void setSparkConfigUpdated(boolean isSparkConfigUpdated) { "Channel logging level for remote Spark driver. One of {DEBUG, ERROR, INFO, TRACE, WARN}."), SPARK_RPC_SASL_MECHANISM("hive.spark.client.rpc.sasl.mechanisms", "DIGEST-MD5", "Name of the SASL mechanism to use for authentication."), + SPARK_LOG_JOBCONF("hive.spark.jobconf.dump", false, "If true, Spark will dump job configuration"), NWAYJOINREORDER("hive.reorder.nway.joins", true, "Runs reordering of tables within single n-way join (i.e.: picks streamtable)"), HIVE_LOG_N_RECORDS("hive.log.every.n.records", 0L, new RangeValidator(0L, null), diff --git a/data/conf/spark/standalone/hive-site.xml b/data/conf/spark/standalone/hive-site.xml index 016f568..4dfb94b 100644 --- a/data/conf/spark/standalone/hive-site.xml +++ b/data/conf/spark/standalone/hive-site.xml @@ -230,4 +230,9 @@ hive_admin_user + + hive.spark.jobconf.dump + true + + diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java index 6701a29..3ad6edd 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.Serializable; +import java.io.StringWriter; import java.net.URI; import java.net.URISyntaxException; import java.util.ArrayList; @@ -33,6 +34,7 @@ import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; @@ -231,6 +233,7 @@ public Serializable call(JobContext jc) throws Exception { Path localScratchDir = KryoSerializer.deserialize(scratchDirBytes, Path.class); SparkWork localSparkWork = KryoSerializer.deserialize(sparkWorkBytes, SparkWork.class); + logConfigurations(localJobConf); SparkCounters sparkCounters = new SparkCounters(jc.sc()); Map> prefixes = localSparkWork.getRequiredCounterPrefix(); @@ -255,6 +258,18 @@ public Serializable call(JobContext jc) throws Exception { jc.monitor(future, sparkCounters, plan.getCachedRDDIds()); return null; } - } + private void logConfigurations(JobConf localJobConf) { + if (localJobConf.getBoolean(HiveConf.ConfVars.SPARK_LOG_JOBCONF.varname, HiveConf.ConfVars.SPARK_LOG_JOBCONF.defaultBoolVal)) { + LOG.info("Logging job configuration: "); + StringWriter outWriter = new StringWriter(); + try { + Configuration.dumpConfiguration(localJobConf, outWriter); + } catch (IOException e) { + LOG.warn("Error logging job configuration", e); + } + LOG.info(outWriter.toString()); + } + } + } }