You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hama.apache.org by tj...@apache.org on 2011/10/11 11:50:03 UTC
svn commit: r1181674 - in /incubator/hama/branches/HamaV2/server:
lib/guava.jar lib/protobuf.jar
src/main/java/org/apache/hama/bsp/YARNBSPJob.java
src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java
Author: tjungblut
Date: Tue Oct 11 09:50:02 2011
New Revision: 1181674
URL: http://svn.apache.org/viewvc?rev=1181674&view=rev
Log:
Added google libs and a yarn job example.
Added:
incubator/hama/branches/HamaV2/server/lib/guava.jar (with props)
incubator/hama/branches/HamaV2/server/lib/protobuf.jar (with props)
incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java (with props)
Modified:
incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YARNBSPJob.java
Added: incubator/hama/branches/HamaV2/server/lib/guava.jar
URL: http://svn.apache.org/viewvc/incubator/hama/branches/HamaV2/server/lib/guava.jar?rev=1181674&view=auto
==============================================================================
Binary file - no diff available.
Propchange: incubator/hama/branches/HamaV2/server/lib/guava.jar
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: incubator/hama/branches/HamaV2/server/lib/protobuf.jar
URL: http://svn.apache.org/viewvc/incubator/hama/branches/HamaV2/server/lib/protobuf.jar?rev=1181674&view=auto
==============================================================================
Binary file - no diff available.
Propchange: incubator/hama/branches/HamaV2/server/lib/protobuf.jar
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Modified: incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YARNBSPJob.java
URL: http://svn.apache.org/viewvc/incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YARNBSPJob.java?rev=1181674&r1=1181673&r2=1181674&view=diff
==============================================================================
--- incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YARNBSPJob.java (original)
+++ incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YARNBSPJob.java Tue Oct 11 09:50:02 2011
@@ -141,6 +141,7 @@ public class YARNBSPJob extends BSPJob {
// the ApplicationMaster in the launched container
Path jarPath = new Path(getWorkingDirectory(), id + "/app.jar");
fs.copyFromLocalFile(this.getLocalPath(this.getJar()), jarPath);
+ LOG.info("Copying app jar to " + jarPath);
conf.set("bsp.jar", jarPath.makeQualified(fs).toString());
FileStatus jarStatus = fs.getFileStatus(jarPath);
LocalResource amJarRsrc = Records.newRecord(LocalResource.class);
@@ -174,18 +175,21 @@ public class YARNBSPJob extends BSPJob {
out.close();
// Construct the command to be executed on the launched container
- String command = "${JAVA_HOME}" + "/bin/java "
+ String command = "${JAVA_HOME}" + "/bin/java -cp " + classPathEnv + " "
+ BSPApplicationMaster.class.getCanonicalName() + " "
+ xmlPath.makeQualified(fs).toString() + " 1>"
+ ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + " 2>"
+ ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr";
+
+ LOG.info("Start command: " + command);
amContainer.setCommands(Collections.singletonList(command));
Resource capability = Records.newRecord(Resource.class);
// we have at least 3 threads, which comsumes 1mb each, for each bsptask and
- // a base usage of 100mb
- capability.setMemory(100 + 3 * this.getNumBspTask());
+ // a base usage of 250mb
+ // TODO this is too large
+ capability.setMemory(2048);
LOG.info("Set memory for the application master to "
+ capability.getMemory() + "mb!");
amContainer.setResource(capability);
@@ -205,6 +209,7 @@ public class YARNBSPJob extends BSPJob {
GetApplicationReportResponse reportResponse = applicationsManager
.getApplicationReport(reportRequest);
report = reportResponse.getApplicationReport();
+ LOG.info("Got report: " + report.getApplicationId() + " " + report.getHost());
submitted = true;
}
Added: incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java
URL: http://svn.apache.org/viewvc/incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java?rev=1181674&view=auto
==============================================================================
--- incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java (added)
+++ incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java Tue Oct 11 09:50:02 2011
@@ -0,0 +1,79 @@
+/**
+ * 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.hama.bsp;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hama.HamaConfiguration;
+import org.apache.zookeeper.KeeperException;
+
+public class YarnSerializePrinting {
+
+ public static class HelloBSP extends BSP {
+ public static final Log LOG = LogFactory.getLog(HelloBSP.class);
+ private Configuration conf;
+ private final static int PRINT_INTERVAL = 1000;
+ private int num;
+
+ @Override
+ public void bsp(BSPPeer bspPeer) throws IOException, KeeperException,
+ InterruptedException {
+
+ LOG.info(bspPeer.getAllPeerNames());
+ int i = 0;
+ for (String otherPeer : bspPeer.getAllPeerNames()) {
+ String peerName = bspPeer.getPeerName();
+ if (peerName.equals(otherPeer)) {
+ LOG.info("Hello BSP from " + (i + 1) + " of " + num + ": " + peerName);
+ }
+
+ Thread.sleep(PRINT_INTERVAL);
+ bspPeer.sync();
+ i++;
+ }
+ }
+
+ @Override
+ public Configuration getConf() {
+ return conf;
+ }
+
+ @Override
+ public void setConf(Configuration conf) {
+ this.conf = conf;
+ }
+
+ public static void main(String[] args) throws IOException, InterruptedException, ClassNotFoundException {
+ HamaConfiguration conf = new HamaConfiguration();
+ // TODO some keys that should be within a conf
+ conf.set("yarn.resourcemanager.address", "0.0.0.0:8040");
+ conf.set("bsp.local.dir","/tmp/bsp-yarn/");
+ YARNBSPJob job = new YARNBSPJob(conf);
+ job.setBspClass(HelloBSP.class);
+ job.setJarByClass(HelloBSP.class);
+ job.setJobName("Serialize Printing");
+ job.setMemoryUsedPerTaskInMb(50);
+ job.setNumBspTask(3);
+ job.waitForCompletion(true);
+ }
+
+ }
+}
Propchange: incubator/hama/branches/HamaV2/server/src/main/java/org/apache/hama/bsp/YarnSerializePrinting.java
------------------------------------------------------------------------------
svn:eol-style = native