forked from apache/incubator-livy
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
LIVY-126. Add integration tests for batches.
Run a couple of simple apps using the batch API. Some adjustment was needed to make things work; mainly, "spark.submit.deployMode" is not supported by the default Spark dependency (1.5.0 from CDH), so we need to use the old style "yarn-cluster" otherwise batches will fail to launch. For the tests to work I also needed explicit access to the HDFS cluster, which means I broke RealCluster in the process. Need to figure out how to implement that kind of functionality when not using a mini cluster. Closes apache#119
- Loading branch information
Marcelo Vanzin
committed
May 2, 2016
1 parent
cf73a66
commit d609977
Showing
9 changed files
with
238 additions
and
10 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
119 changes: 119 additions & 0 deletions
119
integration-test/src/test/scala/com/cloudera/livy/test/BatchIT.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,119 @@ | ||
/* | ||
* Licensed to Cloudera, Inc. under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.livy.test | ||
|
||
import java.io.File | ||
import java.util.UUID | ||
import javax.servlet.http.HttpServletResponse._ | ||
|
||
import scala.concurrent.duration._ | ||
import scala.language.postfixOps | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.FileSystem | ||
import org.apache.hadoop.fs.Path | ||
import org.scalatest.concurrent.Eventually._ | ||
|
||
import com.cloudera.livy.client.common.HttpMessages._ | ||
import com.cloudera.livy.server.batch.CreateBatchRequest | ||
import com.cloudera.livy.sessions.SessionState | ||
import com.cloudera.livy.test.apps._ | ||
import com.cloudera.livy.test.framework.BaseIntegrationTestSuite | ||
|
||
class BatchIT extends BaseIntegrationTestSuite { | ||
|
||
private var testLibPath: Path = _ | ||
private var conf: Configuration = _ | ||
private var fs: FileSystem = _ | ||
|
||
test("upload test lib") { | ||
// Load the HDFS config from the generated path, if available. | ||
// TODO: how to do this for remote clusters. | ||
conf = new Configuration(false) | ||
cluster.configDir().listFiles().foreach { f => | ||
if (f.getName().endsWith(".xml")) { | ||
conf.addResource(new Path(f.toURI())) | ||
} | ||
} | ||
|
||
val hdfsPath = new Path("/testlib-" + UUID.randomUUID().toString() + ".jar") | ||
fs = FileSystem.get(hdfsPath.toUri(), conf) | ||
fs.copyFromLocalFile(new Path(new File(testLib).toURI()), hdfsPath) | ||
testLibPath = fs.makeQualified(hdfsPath) | ||
} | ||
|
||
test("submit spark app") { | ||
assume(testLibPath != null, "Test lib not uploaded.") | ||
val output = "/" + UUID.randomUUID().toString() | ||
val result = runBatch(classOf[SimpleSparkApp], args = List(output)) | ||
assert(result.state === SessionState.Success().toString) | ||
assert(fs.isDirectory(new Path(output))) | ||
} | ||
|
||
test("submit an app that fails") { | ||
assume(testLibPath != null, "Test lib not uploaded.") | ||
val output = "/" + UUID.randomUUID().toString() | ||
val result = runBatch(classOf[FailingApp], args = List(output)) | ||
assert(result.state === SessionState.Error().toString) | ||
|
||
// The file is written to make sure the app actually ran, instead of just failing for | ||
// some other reason. | ||
assert(fs.isFile(new Path(output))) | ||
} | ||
|
||
private def runBatch(klass: Class[_], args: List[String] = Nil): SessionInfo = { | ||
val request = new CreateBatchRequest() | ||
request.file = testLibPath.toString() | ||
request.className = Some(klass.getName()) | ||
request.args = args | ||
request.conf = Map("spark.yarn.maxAppAttempts" -> "1") | ||
|
||
val response = httpClient.preparePost(s"$livyEndpoint/batches") | ||
.setBody(mapper.writeValueAsString(request)) | ||
.execute() | ||
.get() | ||
|
||
assert(response.getStatusCode() === SC_CREATED) | ||
|
||
val batchInfo = mapper.readValue(response.getResponseBodyAsStream(), classOf[SessionInfo]) | ||
|
||
val terminalStates = Set(SessionState.Error(), SessionState.Dead(), SessionState.Success()) | ||
.map(_.toString) | ||
|
||
var finished = false | ||
try { | ||
eventually(timeout(1 minute), interval(1 second)) { | ||
val response2 = httpClient.prepareGet(s"$livyEndpoint/batches/${batchInfo.id}") | ||
.execute() | ||
.get() | ||
assert(response2.getStatusCode() === SC_OK) | ||
val result = mapper.readValue(response2.getResponseBodyAsStream(), classOf[SessionInfo]) | ||
assert(terminalStates.contains(result.state)) | ||
|
||
finished = true | ||
result | ||
} | ||
} finally { | ||
if (!finished) { | ||
httpClient.prepareDelete(s"$livyEndpoint/batches/${batchInfo.id}").execute() | ||
} | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
40 changes: 40 additions & 0 deletions
40
test-lib/src/main/java/com/cloudera/livy/test/apps/FailingApp.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
* Licensed to Cloudera, Inc. under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.livy.test.apps; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
|
||
public class FailingApp { | ||
|
||
public static void main(String[] args) throws Exception { | ||
if (args.length != 1) { | ||
throw new IllegalArgumentException("Missing output path."); | ||
} | ||
String output = args[0]; | ||
|
||
FileSystem fs = FileSystem.get(new Configuration()); | ||
Path out = new Path(output); | ||
fs.create(out).close(); | ||
|
||
throw new IllegalStateException("This app always fails."); | ||
} | ||
|
||
} |
60 changes: 60 additions & 0 deletions
60
test-lib/src/main/java/com/cloudera/livy/test/apps/SimpleSparkApp.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,60 @@ | ||
/* | ||
* Licensed to Cloudera, Inc. under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. Cloudera, Inc. 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 com.cloudera.livy.test.apps; | ||
|
||
import java.util.Arrays; | ||
import java.util.List; | ||
|
||
import scala.Tuple2; | ||
|
||
import org.apache.spark.api.java.JavaPairRDD; | ||
import org.apache.spark.api.java.JavaSparkContext; | ||
import org.apache.spark.api.java.function.PairFunction; | ||
|
||
public class SimpleSparkApp { | ||
|
||
public static void main(String[] args) throws Exception { | ||
if (args.length != 1) { | ||
throw new IllegalArgumentException("Missing output path."); | ||
} | ||
String output = args[0]; | ||
|
||
JavaSparkContext sc = new JavaSparkContext(); | ||
try { | ||
List<String> data = Arrays.asList("the", "quick", "brown", "fox", "jumped", "over", "the", | ||
"lazy", "dog"); | ||
|
||
JavaPairRDD<String, Integer> rdd = sc.parallelize(data, 3) | ||
.mapToPair(new Counter()); | ||
rdd.saveAsTextFile(output); | ||
} finally { | ||
sc.close(); | ||
} | ||
} | ||
|
||
private static class Counter implements PairFunction<String, String, Integer> { | ||
|
||
@Override | ||
public Tuple2<String, Integer> call(String s) throws Exception { | ||
return new Tuple2<>(s, s.length()); | ||
} | ||
|
||
} | ||
|
||
} |