You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hama.apache.org by ed...@apache.org on 2008/10/13 06:56:17 UTC

svn commit: r703915 - in /incubator/hama/trunk: ./ src/java/org/apache/hama/ src/test/org/apache/hama/

Author: edwardyoon
Date: Sun Oct 12 21:56:16 2008
New Revision: 703915

URL: http://svn.apache.org/viewvc?rev=703915&view=rev
Log:
Separate Interface and Implementation for HamaAdmin

Added:
    incubator/hama/trunk/src/java/org/apache/hama/HamaAdminImpl.java
Modified:
    incubator/hama/trunk/CHANGES.txt
    incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java
    incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java
    incubator/hama/trunk/src/java/org/apache/hama/HamaAdmin.java
    incubator/hama/trunk/src/test/org/apache/hama/TestDenseMatrix.java

Modified: incubator/hama/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/CHANGES.txt?rev=703915&r1=703914&r2=703915&view=diff
==============================================================================
--- incubator/hama/trunk/CHANGES.txt (original)
+++ incubator/hama/trunk/CHANGES.txt Sun Oct 12 21:56:16 2008
@@ -25,6 +25,7 @@
     
   IMPROVEMENTS
     
+    HAMA-78: Separate Interface and Implementation for HamaAdmin (edwardyoon)
     HAMA-76: Remove load() since it duplicated with constructor (edwardyoon)
     HAMA-74: set the number of map/reduce task on the shell (samuel)
     HAMA-73: Add setNumMap/ReduceTasks() to HamaConfiguration (edwardyoon)

Modified: incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java?rev=703915&r1=703914&r2=703915&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/AbstractMatrix.java Sun Oct 12 21:56:16 2008
@@ -38,17 +38,13 @@
 public abstract class AbstractMatrix implements Matrix {
   static final Logger LOG = Logger.getLogger(AbstractMatrix.class);
 
-  /** Hama Configuration */
   protected HamaConfiguration config;
-  /** Hbase admin object */
   protected HBaseAdmin admin;
-  /** The name of Matrix */
+  protected String matrixPath;
   protected String matrixName;
-  /** Hbase table object */
   protected HTable table;
-  /** Matrix attribute description */
   protected HTableDescriptor tableDesc;
-  public HamaAdmin hAdmin;
+  public HamaAdmin store;
 
   /**
    * Sets the job configuration
@@ -63,15 +59,15 @@
       LOG.error(e, e);
     }
 
-    hAdmin = new HamaAdmin(conf, admin);
+    store = new HamaAdminImpl(conf, admin);
   }
 
   /**
    * Create matrix space
    */
   protected void create() throws IOException {
-    // It should run only when table doesn't exist. 
-    if (!admin.tableExists(matrixName)) {
+    // It should run only when table doesn't exist.
+    if (!admin.tableExists(matrixPath)) {
       this.tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN));
       this.tableDesc.addFamily(new HColumnDescriptor(Constants.ATTRIBUTE));
 
@@ -152,19 +148,14 @@
 
   /** {@inheritDoc} */
   public String getName() {
-    return (matrixName != null) ? matrixName : null;
+    return (this.matrixName != null) ? this.matrixName : null;
   }
 
   public void close() throws IOException {
-    if (admin.isTableEnabled(matrixName)) {
-      admin.disableTable(matrixName);
-      admin.deleteTable(matrixName);
-    } else {
-      LOG.info("Table doesn't abled");
-    }
+    store.delete(this.matrixName);
   }
 
   public boolean save(String name) throws IOException {
-    return hAdmin.put(this.matrixName, name);
+    return store.save(this.matrixPath, name);
   }
 }

Modified: incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java?rev=703915&r1=703914&r2=703915&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/DenseMatrix.java Sun Oct 12 21:56:16 2008
@@ -64,15 +64,16 @@
   public DenseMatrix(HamaConfiguration conf, String matrixName) {
     try {
       setConfiguration(conf);
-      if(hAdmin.tableExists(matrixName)) {
-        this.matrixName = hAdmin.get(matrixName);
+      this.matrixName = matrixName; 
+      if(store.matrixExists(matrixName)) {
+        this.matrixPath = store.getPath(matrixName);
       } else {
-        this.matrixName = matrixName;
+        this.matrixPath = matrixName;
         tableDesc = new HTableDescriptor(matrixName);
         create();
       }
 
-      table = new HTable(config, this.matrixName);
+      table = new HTable(config, this.matrixPath);
     } catch (Exception e) {
       e.printStackTrace();
     }
@@ -89,15 +90,15 @@
   public DenseMatrix(HamaConfiguration conf, int m, int n, double s) {
     try {
       setConfiguration(conf);
-      matrixName = RandomVariable.randMatrixName();
+      matrixPath = RandomVariable.randMatrixName();
 
-      if (!admin.tableExists(matrixName)) {
-        tableDesc = new HTableDescriptor(matrixName);
+      if (!admin.tableExists(matrixPath)) {
+        tableDesc = new HTableDescriptor(matrixPath);
         tableDesc.addFamily(new HColumnDescriptor(Constants.COLUMN));
         create();
       }
 
-      table = new HTable(config, matrixName);
+      table = new HTable(config, matrixPath);
 
       for (int i = 0; i < m; i++) {
         for (int j = 0; j < n; j++) {

Modified: incubator/hama/trunk/src/java/org/apache/hama/HamaAdmin.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/HamaAdmin.java?rev=703915&r1=703914&r2=703915&view=diff
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/HamaAdmin.java (original)
+++ incubator/hama/trunk/src/java/org/apache/hama/HamaAdmin.java Sun Oct 12 21:56:16 2008
@@ -21,83 +21,35 @@
 
 import java.io.IOException;
 
-import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTable;
-import org.apache.hadoop.hbase.io.BatchUpdate;
-import org.apache.hadoop.hbase.io.Cell;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Admin tool
- */
-public class HamaAdmin {
-  public HamaConfiguration conf;
-  public HBaseAdmin admin;
-  public HTable table;
-
-  public HamaAdmin(HamaConfiguration conf) {
-    this.conf = conf;
-    initialJob();
-  }
-
-  public HamaAdmin(HamaConfiguration conf, HBaseAdmin admin) {
-    this.conf = conf;
-    this.admin = admin;
-    initialJob();
-  }
-
-  public void initialJob() {
-    try {
-      if (!admin.tableExists(Constants.ADMINTABLE)) {
-        HTableDescriptor tableDesc = new HTableDescriptor(Constants.ADMINTABLE);
-        tableDesc.addFamily(new HColumnDescriptor(Constants.PATHCOLUMN));
-        admin.createTable(tableDesc);
-      }
-
-      table = new HTable(conf, Constants.ADMINTABLE);
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-
-  public boolean put(String matrixName, String name) {
-    boolean result = false;
-
-    BatchUpdate update = new BatchUpdate(name);
-    update.put(Constants.PATHCOLUMN, Bytes.toBytes(matrixName));
-    try {
-      table.commit(update);
-      result = true;
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
+public interface HamaAdmin {
 
-    return result;
-  }
+  /**
+   * Saves matrix
+   * 
+   * @param tempName
+   * @param name
+   * @return true if it saved
+   */
+  public boolean save(String tempName, String name);
 
   /**
    * @param name
-   * @return real table name
+   * @return return a physical path of matrix
+   */
+  public String getPath(String name);
+
+  /**
+   * @param matrixName
+   * @return true if matrix is exist
    */
-  public String get(String name) {
-    try {
-      byte[] result = table.get(name, Constants.PATHCOLUMN).getValue();
-      return Bytes.toString(result);
-    } catch (IOException e) {
-      e.printStackTrace();
-      return null;
-    }
-  }
-
-  public boolean tableExists(String matrixName) {
-    try {
-      Cell result = table.get(matrixName, Constants.PATHCOLUMN);
-      return (result == null) ? false : true;
-    } catch (IOException e) {
-      e.printStackTrace();
-      return false;
-    }
-  }
+  public boolean matrixExists(String matrixName);
+
+  /**
+   * Deletes matrix
+   * 
+   * @param matrixName
+   * @throws IOException
+   */
+  public void delete(String matrixName) throws IOException;
+
 }

Added: incubator/hama/trunk/src/java/org/apache/hama/HamaAdminImpl.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/java/org/apache/hama/HamaAdminImpl.java?rev=703915&view=auto
==============================================================================
--- incubator/hama/trunk/src/java/org/apache/hama/HamaAdminImpl.java (added)
+++ incubator/hama/trunk/src/java/org/apache/hama/HamaAdminImpl.java Sun Oct 12 21:56:16 2008
@@ -0,0 +1,115 @@
+/**
+ * Copyright 2007 The Apache Software Foundation
+ *
+ * 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;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.io.BatchUpdate;
+import org.apache.hadoop.hbase.io.Cell;
+import org.apache.hadoop.hbase.util.Bytes;
+
+public class HamaAdminImpl implements HamaAdmin {
+
+  public HamaConfiguration conf;
+  public HBaseAdmin admin;
+  public HTable table;
+
+  public HamaAdminImpl(HamaConfiguration conf) {
+    this.conf = conf;
+    initialJob();
+  }
+
+  public HamaAdminImpl(HamaConfiguration conf, HBaseAdmin admin) {
+    this.conf = conf;
+    this.admin = admin;
+    initialJob();
+  }
+
+  public void initialJob() {
+    try {
+      if (!admin.tableExists(Constants.ADMINTABLE)) {
+        HTableDescriptor tableDesc = new HTableDescriptor(Constants.ADMINTABLE);
+        tableDesc.addFamily(new HColumnDescriptor(Constants.PATHCOLUMN));
+        admin.createTable(tableDesc);
+      }
+
+      table = new HTable(conf, Constants.ADMINTABLE);
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  /**
+   * @param name
+   * @return real table name
+   */
+  public String getPath(String name) {
+    try {
+      byte[] result = table.get(name, Constants.PATHCOLUMN).getValue();
+      return Bytes.toString(result);
+    } catch (IOException e) {
+      e.printStackTrace();
+      return null;
+    }
+  }
+
+  public boolean matrixExists(String matrixName) {
+    try {
+      Cell result = table.get(matrixName, Constants.PATHCOLUMN);
+      return (result == null) ? false : true;
+    } catch (IOException e) {
+      e.printStackTrace();
+      return false;
+    }
+  }
+
+  public boolean save(String tempName, String name) {
+    boolean result = false;
+
+    BatchUpdate update = new BatchUpdate(name);
+    update.put(Constants.PATHCOLUMN, Bytes.toBytes(tempName));
+    try {
+      table.commit(update);
+      result = true;
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+
+    return result;
+  }
+
+  public void delete(String matrixName) throws IOException {
+    String table;
+    if(matrixExists(matrixName)) {
+      table = getPath(matrixName);
+    } else {
+      table = matrixName;
+    }
+    
+    if (admin.isTableEnabled(table)) {
+      admin.disableTable(table);
+      admin.deleteTable(table);
+    }
+  }
+}

Modified: incubator/hama/trunk/src/test/org/apache/hama/TestDenseMatrix.java
URL: http://svn.apache.org/viewvc/incubator/hama/trunk/src/test/org/apache/hama/TestDenseMatrix.java?rev=703915&r1=703914&r2=703915&view=diff
==============================================================================
--- incubator/hama/trunk/src/test/org/apache/hama/TestDenseMatrix.java (original)
+++ incubator/hama/trunk/src/test/org/apache/hama/TestDenseMatrix.java Sun Oct 12 21:56:16 2008
@@ -38,7 +38,8 @@
   private static int SIZE = 10;
   private static Matrix m1;
   private static Matrix m2;
-
+  private final static String SAVE = "save";
+  
   public static Test suite() {
     TestSetup setup = new TestSetup(new TestSuite(TestDenseMatrix.class)) {
       protected void setUp() throws Exception {
@@ -143,15 +144,17 @@
   }
 
   public void testLoadSave() throws IOException {
-    m1.save("udanax");
+    m1.save(SAVE);
     HCluster hCluster = new HCluster();
-    DenseMatrix loadTest = new DenseMatrix(hCluster.conf, "udanax");
+    DenseMatrix loadTest = new DenseMatrix(hCluster.conf, SAVE);
 
     for (int i = 0; i < loadTest.getRows(); i++) {
       for (int j = 0; j < loadTest.getColumns(); j++) {
         assertEquals(m1.get(i, j), loadTest.get(i, j));
       }
     }
+    
+    assertEquals(loadTest.getName(), SAVE);
   }
 
   /**