-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-3595] Respect configured OutputCommitters when calling saveAsHadoopFile #2450
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 3 commits
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 |
|---|---|---|
|
|
@@ -17,17 +17,21 @@ | |
|
|
||
| package org.apache.spark.rdd | ||
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.collection.mutable.HashSet | ||
| import org.apache.hadoop.fs.FileSystem | ||
| import org.apache.hadoop.mapred._ | ||
| import org.apache.hadoop.util.Progressable | ||
|
|
||
| import scala.collection.mutable.{ArrayBuffer, HashSet} | ||
| import scala.util.Random | ||
|
|
||
| import org.scalatest.FunSuite | ||
| import com.google.common.io.Files | ||
| import org.apache.hadoop.mapreduce._ | ||
| import org.apache.hadoop.conf.{Configuration, Configurable} | ||
|
|
||
| import org.apache.spark.SparkContext._ | ||
| import org.apache.hadoop.conf.{Configurable, Configuration} | ||
| import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter => NewOutputCommitter, | ||
| OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, | ||
| TaskAttemptContext => NewTaskAttempContext} | ||
| import org.apache.spark.{Partitioner, SharedSparkContext} | ||
| import org.apache.spark.SparkContext._ | ||
| import org.scalatest.FunSuite | ||
|
|
||
| class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { | ||
| test("aggregateByKey") { | ||
|
|
@@ -467,7 +471,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { | |
| val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) | ||
|
|
||
| // No error, non-configurable formats still work | ||
| pairs.saveAsNewAPIHadoopFile[FakeFormat]("ignored") | ||
| pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored") | ||
|
|
||
| /* | ||
| Check that configurable formats get configured: | ||
|
|
@@ -478,6 +482,15 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { | |
| pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") | ||
| } | ||
|
|
||
| test("saveAsHadoopFile should respect configured output committers") { | ||
| val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) | ||
| val conf = new JobConf(sc.hadoopConfiguration) | ||
|
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. Could this just start with a blank jobConf rather than reading the one from the spark context? |
||
| conf.setOutputCommitter(classOf[FakeOutputCommitter]) | ||
| pairs.saveAsHadoopFile("ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) | ||
| val ran = sys.props.remove("mapred.committer.ran") | ||
|
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 use of system properties here means this test can't run in parallel. It might be good to do things:
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. Agreed, this part's ugly but it seemed like the least invasive way. I also thought about maybe using a ThreadLocal but didn't get too far. |
||
| assert(ran.isDefined, "OutputCommitter was never called") | ||
| } | ||
|
|
||
| test("lookup") { | ||
| val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) | ||
|
|
||
|
|
@@ -621,40 +634,79 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { | |
| and the test will therefore throw InstantiationException when saveAsNewAPIHadoopFile | ||
| tries to instantiate them with Class.newInstance. | ||
| */ | ||
|
|
||
| /* | ||
| * Original Hadoop API | ||
| */ | ||
| class FakeWriter extends RecordWriter[Integer, Integer] { | ||
| override def write(key: Integer, value: Integer): Unit = () | ||
|
|
||
| override def close(reporter: Reporter): Unit = () | ||
| } | ||
|
|
||
| class FakeOutputCommitter() extends OutputCommitter() { | ||
| override def setupJob(jobContext: JobContext): Unit = () | ||
|
|
||
| override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true | ||
|
|
||
| def close(p1: TaskAttemptContext) = () | ||
| override def setupTask(taskContext: TaskAttemptContext): Unit = () | ||
|
|
||
| override def commitTask(taskContext: TaskAttemptContext): Unit = { | ||
| sys.props("mapred.committer.ran") = "true" | ||
| () | ||
| } | ||
|
|
||
| override def abortTask(taskContext: TaskAttemptContext): Unit = () | ||
| } | ||
|
|
||
| class FakeOutputFormat() extends OutputFormat[Integer, Integer]() { | ||
| override def getRecordWriter( | ||
| ignored: FileSystem, | ||
| job: JobConf, name: String, | ||
| progress: Progressable): RecordWriter[Integer, Integer] = { | ||
| new FakeWriter() | ||
| } | ||
|
|
||
| override def checkOutputSpecs(ignored: FileSystem, job: JobConf): Unit = () | ||
| } | ||
|
|
||
| /* | ||
| * New-style Hadoop API | ||
| */ | ||
| class NewFakeWriter extends NewRecordWriter[Integer, Integer] { | ||
|
|
||
| def close(p1: NewTaskAttempContext) = () | ||
|
|
||
| def write(p1: Integer, p2: Integer) = () | ||
|
|
||
| } | ||
|
|
||
| class FakeCommitter extends OutputCommitter { | ||
| def setupJob(p1: JobContext) = () | ||
| class NewFakeCommitter extends NewOutputCommitter { | ||
| def setupJob(p1: NewJobContext) = () | ||
|
|
||
| def needsTaskCommit(p1: TaskAttemptContext): Boolean = false | ||
| def needsTaskCommit(p1: NewTaskAttempContext): Boolean = false | ||
|
|
||
| def setupTask(p1: TaskAttemptContext) = () | ||
| def setupTask(p1: NewTaskAttempContext) = () | ||
|
|
||
| def commitTask(p1: TaskAttemptContext) = () | ||
| def commitTask(p1: NewTaskAttempContext) = () | ||
|
|
||
| def abortTask(p1: TaskAttemptContext) = () | ||
| def abortTask(p1: NewTaskAttempContext) = () | ||
| } | ||
|
|
||
| class FakeFormat() extends OutputFormat[Integer, Integer]() { | ||
| class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { | ||
|
|
||
| def checkOutputSpecs(p1: JobContext) = () | ||
| def checkOutputSpecs(p1: NewJobContext) = () | ||
|
|
||
| def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { | ||
| new FakeWriter() | ||
| def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { | ||
| new NewFakeWriter() | ||
| } | ||
|
|
||
| def getOutputCommitter(p1: TaskAttemptContext): OutputCommitter = { | ||
| new FakeCommitter() | ||
| def getOutputCommitter(p1: NewTaskAttempContext): NewOutputCommitter = { | ||
| new NewFakeCommitter() | ||
| } | ||
| } | ||
|
|
||
| class ConfigTestFormat() extends FakeFormat() with Configurable { | ||
| class ConfigTestFormat() extends NewFakeFormat() with Configurable { | ||
|
|
||
| var setConfCalled = false | ||
| def setConf(p1: Configuration) = { | ||
|
|
@@ -664,7 +716,7 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { | |
|
|
||
| def getConf: Configuration = null | ||
|
|
||
| override def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { | ||
| override def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { | ||
| assert(setConfCalled, "setConf was never called") | ||
| super.getRecordWriter(p1) | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,88 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.examples | ||
|
|
||
| import org.apache.commons.logging.LogFactory | ||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
| import org.apache.hadoop.mapred._ | ||
| import org.apache.spark.{SparkConf, SparkContext} | ||
| import org.apache.spark.SparkContext._ | ||
|
|
||
| /** | ||
| * An OutputCommitter similar to the one used by default for s3:// URLs in EMR. | ||
| */ | ||
| class DirectOutputCommitter extends OutputCommitter { | ||
|
||
| private final val LOG = LogFactory.getLog("org.apache.spark.examples.DirectOutputCommitter") | ||
|
|
||
| override def setupJob(jobContext: JobContext): Unit = { | ||
| LOG.info("Nothing to do in setupJob") | ||
| } | ||
|
|
||
| override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = { | ||
| LOG.info("Nothing to do in needsTaskCommit"); false | ||
| } | ||
|
|
||
| override def setupTask(taskContext: TaskAttemptContext): Unit = { | ||
| LOG.info("Nothing to do in setupTask") | ||
| } | ||
|
|
||
| override def commitTask(taskContext: TaskAttemptContext): Unit = { | ||
| LOG.info("Nothing to do in commitTask") | ||
| } | ||
|
|
||
| override def abortTask(taskContext: TaskAttemptContext): Unit = { | ||
| LOG.info("Nothing to do in abortTask") | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Run word count on text files stored in S3. | ||
| */ | ||
| object AwsTest { | ||
| def main(args: Array[String]) { | ||
| val (ak, sk, in, out) = args match { | ||
| case Array(ak, sk, in, out) ⇒ (ak, sk, in, out) | ||
| case _ ⇒ { | ||
| sys.error("Usage: AwsTest AWS_ACCESS_KEY_ID AWS_SECRET_ACCESS_KEY s3://INPUT s3://OUTPUT") | ||
| } | ||
| } | ||
|
|
||
| val sparkConf = new SparkConf().setAppName("AwsTest") | ||
| val sc = new SparkContext(sparkConf) | ||
|
|
||
| /* | ||
| * Example setup that closely resembles Elastic MapReduce configuration | ||
| */ | ||
| sc.hadoopConfiguration.set("fs.s3.awsAccessKeyId", ak) | ||
| sc.hadoopConfiguration.set("fs.s3.awsSecretAccessKey", sk) | ||
| sc.hadoopConfiguration.set("mapred.output.committer.class", | ||
| "org.apache.spark.examples.DirectOutputCommitter") | ||
| sc.hadoopConfiguration.set("fs.s3.impl", "org.apache.hadoop.fs.s3native.NativeS3FileSystem") | ||
|
|
||
| val file = sc.textFile(in) | ||
| val counts = file.flatMap { line => | ||
| line.split("\\s") | ||
| }.map { word => | ||
| (word, 1) | ||
| }.reduceByKey(_ + _, 10) | ||
|
|
||
| counts.saveAsTextFile(out) | ||
|
|
||
| sc.stop() | ||
| } | ||
| } | ||
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.
For this comment I'd make it more general:
I'm guessing over time we'll run into many formats that require this.