Index: src/java/org/apache/hama/bsp/BSPPeer.java =================================================================== --- src/java/org/apache/hama/bsp/BSPPeer.java (리비전 950897) +++ src/java/org/apache/hama/bsp/BSPPeer.java (작업 사본) @@ -39,7 +39,7 @@ import org.apache.zookeeper.ZooKeeper; import org.apache.zookeeper.ZooDefs.Ids; -public class BSPPeer implements DefaultBSPPeer, Watcher, BSPPeerInterface { +public class BSPPeer implements Watcher, BSPPeerInterface { public static final Log LOG = LogFactory.getLog(BSPPeer.class); protected Configuration conf; @@ -79,7 +79,7 @@ public void reinitialize() { try { - System.out.println(bindAddress + ":" + bindPort); + LOG.debug("reinitialize(): " + bindAddress + ":" + bindPort); server = RPC.getServer(this, bindAddress, bindPort, conf); server.start(); } catch (IOException e) { @@ -237,4 +237,9 @@ return peer; } + + @Override + public String getServerName() { + return this.serverName; + } } Index: src/java/org/apache/hama/bsp/BSPPeerInterface.java =================================================================== --- src/java/org/apache/hama/bsp/BSPPeerInterface.java (리비전 950897) +++ src/java/org/apache/hama/bsp/BSPPeerInterface.java (작업 사본) @@ -17,10 +17,44 @@ */ package org.apache.hama.bsp; +import java.io.Closeable; import java.io.IOException; +import java.net.InetSocketAddress; -public interface BSPPeerInterface extends BSPRPCProtocolVersion { +import org.apache.hama.Constants; +import org.apache.zookeeper.KeeperException; + +public interface BSPPeerInterface extends BSPRPCProtocolVersion, Closeable, Constants { + + /** + * Send a data with a tag to another BSPSlave corresponding to hostname. + * Messages sent by this method are not guaranteed to be received in a sent + * order. + * + * @param hostname + * @param msg + * @throws IOException + */ + public void send(InetSocketAddress hostname, BSPMessage msg) + throws IOException; + public void put(BSPMessage msg) throws IOException; + + /** + * @return the current message + * @throws IOException + */ + public BSPMessage getCurrentMessage() throws IOException; + /** + * Synchronize all of the data in the local queue to other BSP Peers. + * + * @throws InterruptedException + * @throws KeeperException + */ + public void sync() throws IOException, KeeperException, InterruptedException; + public boolean isRunning(); + + public String getServerName(); } Index: src/java/org/apache/hama/bsp/DefaultBSPPeer.java =================================================================== --- src/java/org/apache/hama/bsp/DefaultBSPPeer.java (리비전 950897) +++ src/java/org/apache/hama/bsp/DefaultBSPPeer.java (작업 사본) @@ -1,57 +0,0 @@ -/** - * 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.Closeable; -import java.io.IOException; -import java.net.InetSocketAddress; - -import org.apache.hama.Constants; -import org.apache.zookeeper.KeeperException; - -/** - * - */ -public interface DefaultBSPPeer extends Closeable, Constants { - - /** - * Send a data with a tag to another BSPSlave corresponding to hostname. - * Messages sent by this method are not guaranteed to be received in a sent - * order. - * - * @param hostname - * @param msg - * @throws IOException - */ - public void send(InetSocketAddress hostname, BSPMessage msg) - throws IOException; - - /** - * @return the current message - * @throws IOException - */ - public BSPMessage getCurrentMessage() throws IOException; - - /** - * Synchronize all of the data in the local queue to other BSP Peers. - * - * @throws InterruptedException - * @throws KeeperException - */ - public void sync() throws IOException, KeeperException, InterruptedException; -} Index: src/java/org/apache/hama/util/ClusterUtil.java =================================================================== --- src/java/org/apache/hama/util/ClusterUtil.java (리비전 950897) +++ src/java/org/apache/hama/util/ClusterUtil.java (작업 사본) @@ -3,7 +3,6 @@ import java.io.IOException; import java.util.List; -import org.apache.commons.logging.Log; import org.apache.hadoop.conf.Configuration; import org.apache.hama.bsp.BSPMaster; import org.apache.hama.bsp.GroomServer; Index: src/test/org/apache/hama/bsp/UserInterface.java =================================================================== --- src/test/org/apache/hama/bsp/UserInterface.java (리비전 950897) +++ src/test/org/apache/hama/bsp/UserInterface.java (작업 사본) @@ -73,7 +73,7 @@ } if (pi != 0.0) - System.out.println(pi); + System.out.println(peer.getServerName() + ": " + pi); } }