You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ke...@apache.org on 2012/03/09 05:13:17 UTC

svn commit: r1298699 - in /hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks: Entity.java ReadEntity.java WriteEntity.java

Author: kevinwilfong
Date: Fri Mar  9 04:13:17 2012
New Revision: 1298699

URL: http://svn.apache.org/viewvc?rev=1298699&view=rev
Log:
HIVE-2838. cleanup readentity/writeentity. (namit via kevinwilfong)

Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/Entity.java
Modified:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/ReadEntity.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/Entity.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/Entity.java?rev=1298699&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/Entity.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/Entity.java Fri Mar  9 04:13:17 2012
@@ -0,0 +1,299 @@
+/**
+ * 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.hadoop.hive.ql.hooks;
+
+import java.io.Serializable;
+import java.net.URI;
+import java.util.Map;
+
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.DummyPartition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+/**
+ * This class encapsulates an object that is being read or written to by the
+ * query. This object may be a table, partition, dfs directory or a local
+ * directory.
+ */
+public class Entity implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * The type of the entity.
+   */
+  public static enum Type {
+    TABLE, PARTITION, DUMMYPARTITION, DFS_DIR, LOCAL_DIR
+  };
+
+  /**
+   * The type.
+   */
+  private Type typ;
+
+  /**
+   * The table. This is null if this is a directory.
+   */
+  private Table t;
+
+  /**
+   * The partition.This is null if this object is not a partition.
+   */
+  private Partition p;
+
+  /**
+   * The directory if this is a directory.
+   */
+  private String d;
+
+  /**
+   * This is derived from t and p, but we need to serialize this field to make
+   * sure Entity.hashCode() does not need to recursively read into t and p.
+   */
+  private String name;
+
+  /**
+   * Whether the output is complete or not. For eg, for dynamic partitions, the
+   * complete output may not be known
+   */
+  private boolean complete;
+
+  public boolean isComplete() {
+    return complete;
+  }
+
+  public void setComplete(boolean complete) {
+    this.complete = complete;;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public Type getTyp() {
+    return typ;
+  }
+
+  public void setTyp(Type typ) {
+    this.typ = typ;
+  }
+
+  public Table getT() {
+    return t;
+  }
+
+  public void setT(Table t) {
+    this.t = t;
+  }
+
+  public Partition getP() {
+    return p;
+  }
+
+  public void setP(Partition p) {
+    this.p = p;
+  }
+
+  public String getD() {
+    return d;
+  }
+
+  public void setD(String d) {
+    this.d = d;
+  }
+
+  /**
+   * Only used by serialization.
+   */
+  public Entity() {
+  }
+
+  /**
+   * Constructor for a table.
+   *
+   * @param t
+   *          Table that is read or written to.
+   */
+  public Entity(Table t) {
+    this(t, true);
+  }
+
+  public Entity(Table t, boolean complete) {
+    d = null;
+    p = null;
+    this.t = t;
+    typ = Type.TABLE;
+    name = computeName();
+    this.complete = complete;
+  }
+
+  /**
+   * Constructor for a partition.
+   *
+   * @param p
+   *          Partition that is read or written to.
+   */
+  public Entity(Partition p) {
+    this(p, true);
+  }
+
+  public Entity(Partition p, boolean complete) {
+    d = null;
+    this.p = p;
+    t = p.getTable();
+    typ = Type.PARTITION;
+    name = computeName();
+    this.complete = complete;
+  }
+
+  public Entity(DummyPartition p, boolean complete) {
+    d = null;
+    this.p = p;
+    t = p.getTable();
+    typ = Type.DUMMYPARTITION;
+    name = computeName();
+    this.complete = complete;
+  }
+
+  /**
+   * Constructor for a file.
+   *
+   * @param d
+   *          The name of the directory that is being read or written to.
+   * @param islocal
+   *          Flag to decide whether this directory is local or in dfs.
+   */
+  public Entity(String d, boolean islocal) {
+    this(d, islocal, true);
+  }
+
+  public Entity(String d, boolean islocal, boolean complete) {
+    this.d = d;
+    p = null;
+    t = null;
+    if (islocal) {
+      typ = Type.LOCAL_DIR;
+    } else {
+      typ = Type.DFS_DIR;
+    }
+    name = computeName();
+    this.complete = complete;
+  }
+
+  /**
+   * Get the parameter map of the Entity.
+   */
+  public Map<String, String> getParameters() {
+    if (p != null) {
+      return p.getParameters();
+    } else {
+      return t.getParameters();
+    }
+  }
+
+  /**
+   * Get the type of the entity.
+   */
+  public Type getType() {
+    return typ;
+  }
+
+  /**
+   * Get the location of the entity.
+   */
+  public URI getLocation() throws Exception {
+    if (typ == Type.TABLE) {
+      return t.getDataLocation();
+    }
+
+    if (typ == Type.PARTITION) {
+      return p.getDataLocation();
+    }
+
+    if (typ == Type.DFS_DIR || typ == Type.LOCAL_DIR) {
+      return new URI(d);
+    }
+
+    return null;
+  }
+
+  /**
+   * Get the partition associated with the entity.
+   */
+  public Partition getPartition() {
+    return p;
+  }
+
+  /**
+   * Get the table associated with the entity.
+   */
+  public Table getTable() {
+    return t;
+  }
+
+  /**
+   * toString function.
+   */
+  @Override
+  public String toString() {
+    return name;
+  }
+
+  private String computeName() {
+    switch (typ) {
+    case TABLE:
+      return t.getDbName() + "@" + t.getTableName();
+    case PARTITION:
+      return t.getDbName() + "@" + t.getTableName() + "@" + p.getName();
+    case DUMMYPARTITION:
+      return p.getName();
+    default:
+      return d;
+    }
+  }
+
+  /**
+   * Equals function.
+   */
+  @Override
+  public boolean equals(Object o) {
+    if (o == null) {
+      return false;
+    }
+
+    if (o instanceof Entity) {
+      Entity ore = (Entity) o;
+      return (toString().equalsIgnoreCase(ore.toString()));
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Hashcode function.
+   */
+  @Override
+  public int hashCode() {
+    return toString().hashCode();
+  }
+
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/ReadEntity.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/ReadEntity.java?rev=1298699&r1=1298698&r2=1298699&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/ReadEntity.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/ReadEntity.java Fri Mar  9 04:13:17 2012
@@ -19,8 +19,6 @@
 package org.apache.hadoop.hive.ql.hooks;
 
 import java.io.Serializable;
-import java.net.URI;
-import java.util.Map;
 
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -29,57 +27,13 @@ import org.apache.hadoop.hive.ql.metadat
  * This class encapsulates the information on the partition and tables that are
  * read by the query.
  */
-public class ReadEntity implements Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The table.
-   */
-  private Table t;
-
-  /**
-   * The partition. This is null for a non partitioned table.
-   */
-  private Partition p;
-
-  /**
-   * This is derived from t and p, but we need to serialize this field to make sure
-   * ReadEntity.hashCode() does not need to recursively read into t and p.
-   */
-  private String name;
-
-  public String getName() {
-    if (name == null) {
-      name = computeName();
-    }
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public void setP(Partition p) {
-    this.p = p;
-  }
-
-  public void setT(Table t) {
-    this.t = t;
-  }
-
-  public Partition getP() {
-    return p;
-  }
-
-  public Table getT() {
-    return t;
-  }
+public class ReadEntity extends Entity implements Serializable {
 
   /**
    * For serialization only.
    */
   public ReadEntity() {
+    super();
   }
 
   /**
@@ -89,9 +43,7 @@ public class ReadEntity implements Seria
    *          The Table that the query reads from.
    */
   public ReadEntity(Table t) {
-    this.t = t;
-    p = null;
-    name = null;
+    super(t);
   }
 
   /**
@@ -101,86 +53,7 @@ public class ReadEntity implements Seria
    *          The partition that the query reads from.
    */
   public ReadEntity(Partition p) {
-    t = p.getTable();
-    this.p = p;
-    name = null;
-  }
-
-  private String computeName() {
-    StringBuilder sb = new StringBuilder();
-    if (p != null) {
-      sb.append(p.getTable().getDbName());
-      sb.append('@');
-      sb.append(p.getTable().getTableName());
-      sb.append('@');
-      sb.append(p.getName());
-    } else {
-      sb.append(t.getDbName());
-      sb.append('@');
-      sb.append(t.getTableName());
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Enum that tells what time of a read entity this is.
-   */
-  public static enum Type {
-    TABLE, PARTITION
-  };
-
-  /**
-   * Get the type.
-   */
-  public Type getType() {
-    return p == null ? Type.TABLE : Type.PARTITION;
-  }
-
-  /**
-   * Get the parameter map of the Entity.
-   */
-  public Map<String, String> getParameters() {
-    if (p != null) {
-      return p.getParameters();
-    } else {
-      return t.getParameters();
-    }
-  }
-
-  /**
-   * Get the location of the entity.
-   */
-  public URI getLocation() {
-    if (p != null) {
-      return p.getDataLocation();
-    } else {
-      return t.getDataLocation();
-    }
-  }
-
-  /**
-   * Get partition entity.
-   */
-  public Partition getPartition() {
-    return p;
-  }
-
-  /**
-   * Get table entity.
-   */
-  public Table getTable() {
-    return t;
-  }
-
-  /**
-   * toString function.
-   */
-  @Override
-  public String toString() {
-    if (name == null) {
-      name = computeName();
-    }
-    return name;
+    super(p);
   }
 
   /**
@@ -199,12 +72,4 @@ public class ReadEntity implements Seria
       return false;
     }
   }
-
-  /**
-   * Hashcode function.
-   */
-  @Override
-  public int hashCode() {
-    return toString().hashCode();
-  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java?rev=1298699&r1=1298698&r2=1298699&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/hooks/WriteEntity.java Fri Mar  9 04:13:17 2012
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hive.ql.hooks;
 
 import java.io.Serializable;
-import java.net.URI;
 
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.DummyPartition;
@@ -29,100 +28,13 @@ import org.apache.hadoop.hive.ql.metadat
  * This class encapsulates an object that is being written to by the query. This
  * object may be a table, partition, dfs directory or a local directory.
  */
-public class WriteEntity implements Serializable {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The type of the write entity.
-   */
-  public static enum Type {
-    TABLE, PARTITION, DUMMYPARTITION, DFS_DIR, LOCAL_DIR
-  };
-
-  /**
-   * The type.
-   */
-  private Type typ;
-
-  /**
-   * The table. This is null if this is a directory.
-   */
-  private Table t;
-
-  /**
-   * The partition.This is null if this object is not a partition.
-   */
-  private Partition p;
-
-  /**
-   * The directory if this is a directory.
-   */
-  private String d;
-
-  /**
-   * This is derived from t and p, but we need to serialize this field to make sure
-   * WriteEntity.hashCode() does not need to recursively read into t and p.
-   */
-  private String name;
-
-  /**
-   * Whether the output is complete or not. For eg, in case of dynamic partitions, the complete output
-   * may not be known
-   */
-  private boolean complete;
-
-  public boolean isComplete() {
-    return complete;
-  }
-
-  public void setComplete(boolean complete) {
-    this.complete = complete;;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  public Type getTyp() {
-    return typ;
-  }
-
-  public void setTyp(Type typ) {
-    this.typ = typ;
-  }
-
-  public Table getT() {
-    return t;
-  }
-
-  public void setT(Table t) {
-    this.t = t;
-  }
-
-  public Partition getP() {
-    return p;
-  }
-
-  public void setP(Partition p) {
-    this.p = p;
-  }
-
-  public String getD() {
-    return d;
-  }
-
-  public void setD(String d) {
-    this.d = d;
-  }
+public class WriteEntity extends Entity implements Serializable {
 
   /**
    * Only used by serialization.
    */
   public WriteEntity() {
+    super();
   }
 
   /**
@@ -136,12 +48,7 @@ public class WriteEntity implements Seri
   }
 
   public WriteEntity(Table t, boolean complete) {
-    d = null;
-    p = null;
-    this.t = t;
-    typ = Type.TABLE;
-    name = computeName();
-    this.complete = complete;
+    super(t, complete);
   }
 
   /**
@@ -155,21 +62,11 @@ public class WriteEntity implements Seri
   }
 
   public WriteEntity(Partition p, boolean complete) {
-    d = null;
-    this.p = p;
-    t = p.getTable();
-    typ = Type.PARTITION;
-    name = computeName();
-    this.complete = complete;
+    super(p, complete);
   }
 
   public WriteEntity(DummyPartition p, boolean complete) {
-    d = null;
-    this.p = p;
-    t = p.getTable();
-    typ = Type.DUMMYPARTITION;
-    name = computeName();
-    this.complete = complete;
+    super(p, complete);
   }
 
   /**
@@ -185,77 +82,7 @@ public class WriteEntity implements Seri
   }
 
   public WriteEntity(String d, boolean islocal, boolean complete) {
-    this.d = d;
-    p = null;
-    t = null;
-    if (islocal) {
-      typ = Type.LOCAL_DIR;
-    } else {
-      typ = Type.DFS_DIR;
-    }
-    name = computeName();
-    this.complete = complete;
-  }
-
-  /**
-   * Get the type of the entity.
-   */
-  public Type getType() {
-    return typ;
-  }
-
-  /**
-   * Get the location of the entity.
-   */
-  public URI getLocation() throws Exception {
-    if (typ == Type.TABLE) {
-      return t.getDataLocation();
-    }
-
-    if (typ == Type.PARTITION) {
-      return p.getDataLocation();
-    }
-
-    if (typ == Type.DFS_DIR || typ == Type.LOCAL_DIR) {
-      return new URI(d);
-    }
-
-    return null;
-  }
-
-  /**
-   * Get the partition associated with the entity.
-   */
-  public Partition getPartition() {
-    return p;
-  }
-
-  /**
-   * Get the table associated with the entity.
-   */
-  public Table getTable() {
-    return t;
-  }
-
-  /**
-   * toString function.
-   */
-  @Override
-  public String toString() {
-    return name;
-  }
-
-  private String computeName() {
-    switch (typ) {
-    case TABLE:
-      return t.getDbName() + "@" + t.getTableName();
-    case PARTITION:
-      return t.getDbName() + "@" + t.getTableName() + "@" + p.getName();
-    case DUMMYPARTITION:
-      return p.getName();
-    default:
-      return d;
-    }
+    super(d, islocal, complete);
   }
 
   /**
@@ -274,13 +101,4 @@ public class WriteEntity implements Seri
       return false;
     }
   }
-
-  /**
-   * Hashcode function.
-   */
-  @Override
-  public int hashCode() {
-    return toString().hashCode();
-  }
-
 }