-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-4229] Create hadoop configuration in a consistent way #3543
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
Changes from all commits
c41a4b4
b48ad63
413f916
7a20953
bfc550e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration | |
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.ScalaReflection | ||
| import org.apache.spark.sql.catalyst.analysis._ | ||
|
|
@@ -262,7 +263,7 @@ class SQLContext(@transient val sparkContext: SparkContext) | |
| def createParquetFile[A <: Product : TypeTag]( | ||
| path: String, | ||
| allowExisting: Boolean = true, | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think this should be using the "hadoopConfiguration" object in the SparkContext. That has all the hadoop related configuration already setup and should be what is automatically used. @marmbrus should have a better idea.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I seem to recall there being potential thread safety issues related to Quick search turned up e.g. https://issues.apache.org/jira/browse/SPARK-2546 I'm not sure how relevant that is to all of these existing situations where On Tue, Dec 9, 2014 at 5:07 PM, Tathagata Das [email protected]
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @koeninger The issue that you linked is concerned with thread-safety issues when multiple threads concurrently modify the same It turns out that there's another, older thread-safety issue related to
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So let me see if I have things straight
So.... Use tdas' suggestion, add a HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized block to SparkHadoopUtil.newConfiguration? And people are out of luck if they have code that used to work because they were modifying new blank instances of Configuration, rather than the now-shared one?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we're going to use Regarding whether sc.hadoopConfiguration.set("es.resource", "syslog/entry")
output.saveAsHadoopFile[ESOutputFormat]("-")In Spark 1.x, I don't think we'll be able to safely transition away from using the shared However, I think that there's much less risk of running into thread-safety issues as a result of this. It seems fairly unlikely that you'll have multiple threads mutating the shared configuration in the driver JVM. In executor JVMs, most Hadoop In a nutshell, I don't think that the shared
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I don't think that users were able to access the old |
||
| conf: Configuration = new Configuration()): SchemaRDD = { | ||
| conf: Configuration = sparkContext.hadoopConfiguration): SchemaRDD = { | ||
| new SchemaRDD( | ||
| this, | ||
| ParquetRelation.createEmpty( | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration | |
|
|
||
| import org.apache.spark.annotation.{DeveloperApi, Experimental} | ||
| import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.sql.{SQLContext, StructType => SStructType} | ||
| import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType | ||
| import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} | ||
|
|
@@ -84,7 +85,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { | |
| beanClass: Class[_], | ||
| path: String, | ||
| allowExisting: Boolean = true, | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same comment as I made in SQLContext |
||
| conf: Configuration = new Configuration()): JavaSchemaRDD = { | ||
| conf: Configuration = sqlContext.sparkContext.hadoopConfiguration): JavaSchemaRDD = { | ||
| new JavaSchemaRDD( | ||
| sqlContext, | ||
| ParquetRelation.createEmpty(path, getSchema(beanClass), allowExisting, conf, sqlContext)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,6 +32,7 @@ import org.apache.hadoop.io.{LongWritable, Text} | |
| import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} | ||
| import org.apache.hadoop.mapreduce.lib.input.TextInputFormat | ||
| import org.apache.spark._ | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.streaming.dstream._ | ||
|
|
@@ -104,7 +105,7 @@ class StreamingContext private[streaming] ( | |
| * Recreate a StreamingContext from a checkpoint file. | ||
| * @param path Path to the directory that was specified as the checkpoint directory | ||
| */ | ||
| def this(path: String) = this(path, new Configuration) | ||
| def this(path: String) = this(path, SparkHadoopUtil.get.conf) | ||
|
|
||
| if (sc_ == null && cp_ == null) { | ||
| throw new Exception("Spark Streaming cannot be initialized with " + | ||
|
|
@@ -545,7 +546,7 @@ object StreamingContext extends Logging { | |
| def getOrCreate( | ||
| checkpointPath: String, | ||
| creatingFunc: () => StreamingContext, | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I approve this change. |
||
| hadoopConf: Configuration = new Configuration(), | ||
| hadoopConf: Configuration = SparkHadoopUtil.get.conf, | ||
| createOnError: Boolean = false | ||
| ): StreamingContext = { | ||
| val checkpointOption = try { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,6 +33,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaUtils} | |
| import org.apache.spark.api.java.JavaPairRDD._ | ||
| import org.apache.spark.api.java.JavaSparkContext.fakeClassTag | ||
| import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.streaming._ | ||
|
|
@@ -789,7 +790,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( | |
| keyClass: Class[_], | ||
| valueClass: Class[_], | ||
| outputFormatClass: Class[_ <: NewOutputFormat[_, _]], | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should also be the configuration from the
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The scope of this PR is pretty wide in terms of the number of classes it touches, causing issues as different places needs to be handled differently. If you considered moving this sort of changes (
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Based on what Marcelo Vanzin said on the dev list when I brought this issue I agree it's used in a lot of different places, but I'm not sure how On Sun, Jan 4, 2015 at 6:28 AM, Tathagata Das [email protected]
|
||
| conf: Configuration = new Configuration) { | ||
| conf: Configuration = dstream.context.sparkContext.hadoopConfiguration) { | ||
| dstream.saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, outputFormatClass, conf) | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Unrelated to this PR?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pretty sure that's just diff getting confused based on where the hadoop doc changes were inserted, same lines are marked as removed lower in the diff