@@ -349,25 +349,27 @@ class SparkContext(
349349 * other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
350350 * etc).
351351 */
352- def hadoopRDD [K , V ](
352+ def hadoopRDD [K : ClassTag , V : ClassTag ](
353353 conf : JobConf ,
354354 inputFormatClass : Class [_ <: InputFormat [K , V ]],
355355 keyClass : Class [K ],
356356 valueClass : Class [V ],
357- minSplits : Int = defaultMinSplits
357+ minSplits : Int = defaultMinSplits,
358+ cloneKeyValues : Boolean = true
358359 ): RDD [(K , V )] = {
359360 // Add necessary security credentials to the JobConf before broadcasting it.
360361 SparkHadoopUtil .get.addCredentials(conf)
361- new HadoopRDD (this , conf, inputFormatClass, keyClass, valueClass, minSplits)
362+ new HadoopRDD (this , conf, inputFormatClass, keyClass, valueClass, minSplits, cloneKeyValues )
362363 }
363364
364365 /** Get an RDD for a Hadoop file with an arbitrary InputFormat */
365- def hadoopFile [K , V ](
366+ def hadoopFile [K : ClassTag , V : ClassTag ](
366367 path : String ,
367368 inputFormatClass : Class [_ <: InputFormat [K , V ]],
368369 keyClass : Class [K ],
369370 valueClass : Class [V ],
370- minSplits : Int = defaultMinSplits
371+ minSplits : Int = defaultMinSplits,
372+ cloneKeyValues : Boolean = true
371373 ): RDD [(K , V )] = {
372374 // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it.
373375 val confBroadcast = broadcast(new SerializableWritable (hadoopConfiguration))
@@ -379,7 +381,8 @@ class SparkContext(
379381 inputFormatClass,
380382 keyClass,
381383 valueClass,
382- minSplits)
384+ minSplits,
385+ cloneKeyValues)
383386 }
384387
385388 /**
@@ -390,14 +393,15 @@ class SparkContext(
390393 * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path, minSplits)
391394 * }}}
392395 */
393- def hadoopFile [K , V , F <: InputFormat [K , V ]]( path : String , minSplits : Int )
394- (implicit km : ClassTag [ K ], vm : ClassTag [ V ], fm : ClassTag [ F ] )
395- : RDD [(K , V )] = {
396+ def hadoopFile [K , V , F <: InputFormat [K , V ]]
397+ (path : String , minSplits : Int , cloneKeyValues : Boolean = true )
398+ ( implicit km : ClassTag [ K ], vm : ClassTag [ V ], fm : ClassTag [ F ]) : RDD [(K , V )] = {
396399 hadoopFile(path,
397- fm.runtimeClass.asInstanceOf [Class [F ]],
398- km.runtimeClass.asInstanceOf [Class [K ]],
399- vm.runtimeClass.asInstanceOf [Class [V ]],
400- minSplits)
400+ fm.runtimeClass.asInstanceOf [Class [F ]],
401+ km.runtimeClass.asInstanceOf [Class [K ]],
402+ vm.runtimeClass.asInstanceOf [Class [V ]],
403+ minSplits,
404+ cloneKeyValues = cloneKeyValues)
401405 }
402406
403407 /**
@@ -408,61 +412,67 @@ class SparkContext(
408412 * val file = sparkContext.hadoopFile[LongWritable, Text, TextInputFormat](path)
409413 * }}}
410414 */
411- def hadoopFile [K , V , F <: InputFormat [K , V ]](path : String )
415+ def hadoopFile [K , V , F <: InputFormat [K , V ]](path : String , cloneKeyValues : Boolean = true )
412416 (implicit km : ClassTag [K ], vm : ClassTag [V ], fm : ClassTag [F ]): RDD [(K , V )] =
413- hadoopFile[K , V , F ](path, defaultMinSplits)
417+ hadoopFile[K , V , F ](path, defaultMinSplits, cloneKeyValues )
414418
415419 /** Get an RDD for a Hadoop file with an arbitrary new API InputFormat. */
416- def newAPIHadoopFile [K , V , F <: NewInputFormat [K , V ]](path : String )
420+ def newAPIHadoopFile [K , V , F <: NewInputFormat [K , V ]]
421+ (path : String , cloneKeyValues : Boolean = true )
417422 (implicit km : ClassTag [K ], vm : ClassTag [V ], fm : ClassTag [F ]): RDD [(K , V )] = {
418423 newAPIHadoopFile(
419- path,
420- fm.runtimeClass.asInstanceOf [Class [F ]],
421- km.runtimeClass.asInstanceOf [Class [K ]],
422- vm.runtimeClass.asInstanceOf [Class [V ]])
424+ path,
425+ fm.runtimeClass.asInstanceOf [Class [F ]],
426+ km.runtimeClass.asInstanceOf [Class [K ]],
427+ vm.runtimeClass.asInstanceOf [Class [V ]],
428+ cloneKeyValues = cloneKeyValues)
423429 }
424430
425431 /**
426432 * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
427433 * and extra configuration options to pass to the input format.
428434 */
429- def newAPIHadoopFile [K , V , F <: NewInputFormat [K , V ]](
435+ def newAPIHadoopFile [K : ClassTag , V : ClassTag , F <: NewInputFormat [K , V ]](
430436 path : String ,
431437 fClass : Class [F ],
432438 kClass : Class [K ],
433439 vClass : Class [V ],
434- conf : Configuration = hadoopConfiguration): RDD [(K , V )] = {
440+ conf : Configuration = hadoopConfiguration,
441+ cloneKeyValues : Boolean = true ): RDD [(K , V )] = {
435442 val job = new NewHadoopJob (conf)
436443 NewFileInputFormat .addInputPath(job, new Path (path))
437444 val updatedConf = job.getConfiguration
438- new NewHadoopRDD (this , fClass, kClass, vClass, updatedConf)
445+ new NewHadoopRDD (this , fClass, kClass, vClass, updatedConf, cloneKeyValues )
439446 }
440447
441448 /**
442449 * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
443450 * and extra configuration options to pass to the input format.
444451 */
445- def newAPIHadoopRDD [K , V , F <: NewInputFormat [K , V ]](
452+ def newAPIHadoopRDD [K : ClassTag , V : ClassTag , F <: NewInputFormat [K , V ]](
446453 conf : Configuration = hadoopConfiguration,
447454 fClass : Class [F ],
448455 kClass : Class [K ],
449- vClass : Class [V ]): RDD [(K , V )] = {
450- new NewHadoopRDD (this , fClass, kClass, vClass, conf)
456+ vClass : Class [V ],
457+ cloneKeyValues : Boolean = true ): RDD [(K , V )] = {
458+ new NewHadoopRDD (this , fClass, kClass, vClass, conf, cloneKeyValues)
451459 }
452460
453461 /** Get an RDD for a Hadoop SequenceFile with given key and value types. */
454- def sequenceFile [K , V ](path : String ,
462+ def sequenceFile [K : ClassTag , V : ClassTag ](path : String ,
455463 keyClass : Class [K ],
456464 valueClass : Class [V ],
457- minSplits : Int
465+ minSplits : Int ,
466+ cloneKeyValues : Boolean = true
458467 ): RDD [(K , V )] = {
459468 val inputFormatClass = classOf [SequenceFileInputFormat [K , V ]]
460- hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits)
469+ hadoopFile(path, inputFormatClass, keyClass, valueClass, minSplits, cloneKeyValues )
461470 }
462471
463472 /** Get an RDD for a Hadoop SequenceFile with given key and value types. */
464- def sequenceFile [K , V ](path : String , keyClass : Class [K ], valueClass : Class [V ]): RDD [(K , V )] =
465- sequenceFile(path, keyClass, valueClass, defaultMinSplits)
473+ def sequenceFile [K : ClassTag , V : ClassTag ](path : String , keyClass : Class [K ], valueClass : Class [V ],
474+ cloneKeyValues : Boolean = true ): RDD [(K , V )] =
475+ sequenceFile(path, keyClass, valueClass, defaultMinSplits, cloneKeyValues)
466476
467477 /**
468478 * Version of sequenceFile() for types implicitly convertible to Writables through a
@@ -480,16 +490,16 @@ class SparkContext(
480490 * for the appropriate type. In addition, we pass the converter a ClassTag of its type to
481491 * allow it to figure out the Writable class to use in the subclass case.
482492 */
483- def sequenceFile [K , V ](path : String , minSplits : Int = defaultMinSplits)
484- (implicit km : ClassTag [K ], vm : ClassTag [V ],
493+ def sequenceFile [K , V ](path : String , minSplits : Int = defaultMinSplits,
494+ cloneKeyValues : Boolean = true ) (implicit km : ClassTag [K ], vm : ClassTag [V ],
485495 kcf : () => WritableConverter [K ], vcf : () => WritableConverter [V ])
486496 : RDD [(K , V )] = {
487497 val kc = kcf()
488498 val vc = vcf()
489499 val format = classOf [SequenceFileInputFormat [Writable , Writable ]]
490500 val writables = hadoopFile(path, format,
491501 kc.writableClass(km).asInstanceOf [Class [Writable ]],
492- vc.writableClass(vm).asInstanceOf [Class [Writable ]], minSplits)
502+ vc.writableClass(vm).asInstanceOf [Class [Writable ]], minSplits, cloneKeyValues )
493503 writables.map{case (k,v) => (kc.convert(k), vc.convert(v))}
494504 }
495505
0 commit comments