You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2013/07/02 16:16:33 UTC

[39/51] [partial] TAJO-22: The package prefix should be org.apache.tajo. (DaeMyung Kang via hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/QueryUnitId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/QueryUnitId.java b/tajo-common/src/main/java/tajo/QueryUnitId.java
deleted file mode 100644
index 40096f4..0000000
--- a/tajo-common/src/main/java/tajo/QueryUnitId.java
+++ /dev/null
@@ -1,177 +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 tajo;
-
-import com.google.common.base.Objects;
-import tajo.TajoIdProtos.QueryUnitIdProto;
-import tajo.TajoIdProtos.QueryUnitIdProtoOrBuilder;
-import tajo.common.ProtoObject;
-import tajo.util.TajoIdUtils;
-
-import java.text.NumberFormat;
-
-public class QueryUnitId implements Comparable<QueryUnitId>,
-  ProtoObject<QueryUnitIdProto> {
-  private static final String PREFIX = "t";
-
-  static final ThreadLocal<NumberFormat> queryUnitIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(6);
-          return fmt;
-        }
-      };
-  
-  private SubQueryId subQueryId = null;
-  private int id = -1;
-  private String finalId = null;
-  
-  private QueryUnitIdProto proto = QueryUnitIdProto.getDefaultInstance();
-  private QueryUnitIdProto.Builder builder = null;
-  private boolean viaProto = false;
-  
-  public QueryUnitId() {
-    builder = QueryUnitIdProto.newBuilder();
-  }
-  
-  public QueryUnitId(final SubQueryId subQueryId,
-      final int id) {
-    this.subQueryId = subQueryId;
-    this.id = id;
-  }
-  
-  public QueryUnitId(QueryUnitIdProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-  
-  public QueryUnitId(final String finalId) {
-    this.finalId = finalId;
-    int i = finalId.lastIndexOf(QueryId.SEPARATOR);
-    this.subQueryId = TajoIdUtils.newSubQueryId(finalId.substring(0, i));
-    this.id = Integer.valueOf(finalId.substring(i+1));
-  }
-  
-  public int getId() {
-    QueryUnitIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.id != -1) {
-      return this.id;
-    }
-    if (!p.hasId()) {
-      return -1;
-    }
-    this.id = p.getId();
-    return id;
-  }
-  
-  public SubQueryId getSubQueryId() {
-    QueryUnitIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.subQueryId != null) {
-      return this.subQueryId;
-    }
-    if (!p.hasSubQueryId()) {
-      return null;
-    }
-    this.subQueryId = TajoIdUtils.newSubQueryId(p.getSubQueryId());
-    return this.subQueryId;
-  }
-  
-  public QueryId getQueryId() {
-    return this.getSubQueryId().getQueryId();
-  }
-  
-  @Override
-  public final String toString() {
-    if (finalId == null) {
-      StringBuilder sb = new StringBuilder(PREFIX);
-      QueryId appId = getSubQueryId().getQueryId();
-      sb.append(QueryId.SEPARATOR).append(
-          appId.getApplicationId().getClusterTimestamp())
-      .append(QueryId.SEPARATOR).append(
-          QueryId.appIdFormat.get().format(appId.getApplicationId().getId()))
-      .append(QueryId.SEPARATOR).append(
-          QueryId.attemptIdFormat.get().format(appId.getAttemptId()))
-      .append(QueryId.SEPARATOR).append(
-          SubQueryId.subQueryIdFormat.get().format(getSubQueryId().getId()))
-      .append(QueryId.SEPARATOR).append(queryUnitIdFormat.get().format(getId()));
-      finalId = sb.toString();
-    }
-    return this.finalId;
-  }
-  
-  @Override
-  public final boolean equals(final Object o) {
-    if (o instanceof QueryUnitId) {
-      QueryUnitId other = (QueryUnitId) o;
-      return getSubQueryId().equals(other.getSubQueryId()) &&
-          getId() == other.getId();
-    }    
-    return false;
-  }
-  
-  @Override
-  public int hashCode() {
-    return Objects.hashCode(getSubQueryId(), getId());
-  }
-
-  @Override
-  public final int compareTo(final QueryUnitId o) {
-    return this.toString().compareTo(o.toString());
-  }
-  
-  private void mergeProtoToLocal() {
-    QueryUnitIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (subQueryId == null) {
-      subQueryId = TajoIdUtils.newSubQueryId(p.getSubQueryId());
-    }
-    if (id == -1) {
-      id = p.getId();
-    }
-  }
-
-  @Override
-  public void initFromProto() {
-    mergeProtoToLocal();
-  }
-  
-  private void mergeLocalToBuilder() {
-    if (builder == null) {
-      builder = QueryUnitIdProto.newBuilder(proto);
-    }
-    if (this.subQueryId != null) {
-      builder.setSubQueryId(subQueryId.getProto());
-    }
-    if (this.id != -1) {
-      builder.setId(id);
-    }
-  }
-
-  @Override
-  public QueryUnitIdProto getProto() {
-    if (!viaProto) {
-      mergeLocalToBuilder();
-      proto = builder.build();
-      viaProto = true;
-    }
-    return proto;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/SubQueryId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/SubQueryId.java b/tajo-common/src/main/java/tajo/SubQueryId.java
deleted file mode 100644
index 5666059..0000000
--- a/tajo-common/src/main/java/tajo/SubQueryId.java
+++ /dev/null
@@ -1,166 +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 tajo;
-
-import tajo.TajoIdProtos.SubQueryIdProto;
-import tajo.TajoIdProtos.SubQueryIdProtoOrBuilder;
-
-import java.text.NumberFormat;
-
-public class SubQueryId implements Comparable<SubQueryId> {
-  public static final String PREFIX = "sq";
-
-  static final ThreadLocal<NumberFormat> subQueryIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(2);
-          return fmt;
-        }
-      };
-
-  private SubQueryIdProto proto = SubQueryIdProto.getDefaultInstance();
-  private SubQueryIdProto.Builder builder = null;
-  private boolean viaProto = false;
-
-  private QueryId queryId = null;
-
-  public SubQueryId() {
-    builder = SubQueryIdProto.newBuilder(proto);
-  }
-
-  public SubQueryId(SubQueryIdProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-
-  /**
-   * @return the subquery number.
-   */
-  public synchronized int getId() {
-    SubQueryIdProtoOrBuilder p = viaProto ? proto : builder;
-    return (p.getId());
-  }
-
-  public synchronized void setId(int id) {
-    maybeInitBuilder();
-    builder.setId((id));
-  }
-
-  /**
-   * @return the associated <code>QueryId</code>
-   */
-  public synchronized QueryId getQueryId() {
-    SubQueryIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.queryId != null) {
-      return this.queryId;
-    }
-    if (!p.hasQueryId()) {
-      return null;
-    }
-    queryId = new QueryId(p.getQueryId());
-    return queryId;
-  }
-
-  public synchronized void setQueryId(QueryId queryId) {
-    maybeInitBuilder();
-    if (queryId == null)
-      builder.clearQueryId();
-    this.queryId = queryId;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + getId();
-    result = prime * result + getQueryId().hashCode();
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    SubQueryId other = (SubQueryId) obj;
-    if (getId() != other.getId())
-      return false;
-    if (!getQueryId().equals(other.getQueryId()))
-      return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder(PREFIX);
-    QueryId queryId = getQueryId();
-    builder.append(QueryId.SEPARATOR).append(queryId.getApplicationId().getClusterTimestamp());
-    builder.append(QueryId.SEPARATOR).append(
-        QueryId.appIdFormat.get().format(queryId.getApplicationId().getId()));
-    builder.append(QueryId.SEPARATOR).append(QueryId.attemptIdFormat.get().format(queryId.getAttemptId()))
-        .append(QueryId.SEPARATOR)
-    .append(subQueryIdFormat.get().format(getId()));
-    return builder.toString();
-  }
-
-  @Override
-  public int compareTo(SubQueryId other) {
-    int queryIdComp = this.getQueryId().compareTo(other.getQueryId());
-    if (queryIdComp == 0) {
-      return this.getId() - other.getId();
-    } else {
-      return queryIdComp;
-    }
-  }
-
-  public synchronized SubQueryIdProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  private synchronized void mergeLocalToBuilder() {
-    if (this.queryId != null
-        && !this.queryId.getProto().equals(builder.getQueryId())) {
-      builder.setQueryId(queryId.getProto());
-    }
-  }
-
-  private synchronized void mergeLocalToProto() {
-    if (viaProto)
-      maybeInitBuilder();
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private synchronized void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = SubQueryIdProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/TajoConstants.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/TajoConstants.java b/tajo-common/src/main/java/tajo/TajoConstants.java
deleted file mode 100644
index e3d339e..0000000
--- a/tajo-common/src/main/java/tajo/TajoConstants.java
+++ /dev/null
@@ -1,28 +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 tajo;
-
-public class TajoConstants {
-
-  public static String WAREHOUSE_DIR = "warehouse";
-
-  public static String USER_QUERYDIR_PREFIX = "tajo";
-
-  private TajoConstants() {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/common/ProtoObject.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/common/ProtoObject.java b/tajo-common/src/main/java/tajo/common/ProtoObject.java
deleted file mode 100644
index 7285a54..0000000
--- a/tajo-common/src/main/java/tajo/common/ProtoObject.java
+++ /dev/null
@@ -1,26 +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 tajo.common;
-
-import com.google.protobuf.Message;
-
-public interface ProtoObject<P extends Message> {
-	public void initFromProto();
-	public P getProto();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/common/Sleeper.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/common/Sleeper.java b/tajo-common/src/main/java/tajo/common/Sleeper.java
deleted file mode 100644
index b71e764..0000000
--- a/tajo-common/src/main/java/tajo/common/Sleeper.java
+++ /dev/null
@@ -1,42 +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 tajo.common;
-
-public class Sleeper {
-  private long before;
-  private long cur;
-
-  public Sleeper() {
-    before = -1;
-  }
-
-  public void sleep(long time) throws InterruptedException {
-    long sleeptime;
-    cur = System.currentTimeMillis();
-    if (before == -1) {
-      sleeptime = time;
-    } else {
-      sleeptime = time - (cur - before);
-    }
-    if (sleeptime > 0) {
-      Thread.sleep(sleeptime);
-    }
-    before = System.currentTimeMillis();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/common/exception/InvalidAddressException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/common/exception/InvalidAddressException.java b/tajo-common/src/main/java/tajo/common/exception/InvalidAddressException.java
deleted file mode 100644
index 3bb7286..0000000
--- a/tajo-common/src/main/java/tajo/common/exception/InvalidAddressException.java
+++ /dev/null
@@ -1,25 +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 tajo.common.exception;
-
-public class InvalidAddressException extends Exception {
-
-	private static final long serialVersionUID = -7266998886109689882L;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/common/exception/NotImplementedException.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/common/exception/NotImplementedException.java b/tajo-common/src/main/java/tajo/common/exception/NotImplementedException.java
deleted file mode 100644
index 3c5f7cd..0000000
--- a/tajo-common/src/main/java/tajo/common/exception/NotImplementedException.java
+++ /dev/null
@@ -1,49 +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 tajo.common.exception;
-
-public class NotImplementedException extends RuntimeException {
-
-  private static final long serialVersionUID = 8515328809349325243L;
-
-  public NotImplementedException() {
-  }
-
-  /**
-   * @param message
-   */
-  public NotImplementedException(String message) {
-    super(message);
-  }
-
-  /**
-   * @param cause
-   */
-  public NotImplementedException(Throwable cause) {
-    super(cause);
-  }
-
-  /**
-   * @param message
-   * @param cause
-   */
-  public NotImplementedException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/common/type/IPv4.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/common/type/IPv4.java b/tajo-common/src/main/java/tajo/common/type/IPv4.java
deleted file mode 100644
index 761cc31..0000000
--- a/tajo-common/src/main/java/tajo/common/type/IPv4.java
+++ /dev/null
@@ -1,181 +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 tajo.common.type;
-
-import org.apache.hadoop.io.Writable;
-import tajo.common.exception.InvalidAddressException;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.StringTokenizer;
-
-public class IPv4 implements Writable, Comparable<IPv4> {
-	
-	public static IPv4 getSubnetMask(int len) {
-		byte[] bytes = new byte[4];
-		int index = 0;
-		while (len > 7) {
-			len -= 8;
-			bytes[index++] = (byte)0xFF;
-		}
-		bytes[index] = (byte)((0xFF >> (8-len)) << (8-len));
-		return new IPv4(bytes);
-	}
-	
-	private byte[] ipBytes;
-	
-	public IPv4() {
-		this.ipBytes = new byte[4];
-	}
-	
-	public IPv4(byte[] bytes) {
-		this.ipBytes = new byte[4];
-		set(bytes);
-	}
-	
-	public IPv4(String ipAddress) throws InvalidAddressException {
-		this.ipBytes = new byte[4];
-		this.set(ipAddress);
-	}
-	
-	public void set(String ipAddress) throws InvalidAddressException {
-		StringTokenizer tokenizer = new StringTokenizer(ipAddress);
-		String token;
-		for (int i = 0; i < 4; i++) {
-			token = tokenizer.nextToken(".");
-			if (token == null) {
-				throw new InvalidAddressException();
-			} else if (Integer.valueOf(token) < 0 || Integer.valueOf(token) > 255) {
-				throw new InvalidAddressException();
-			}
-			//			ipBytes[i] = Short.valueOf(token).byteValue();
-			this.ipBytes[i] = (byte)(((Integer.valueOf(token) << 24) >> 24) & 0xFF);
-		}
-	}
-	
-	public void set(byte[] bytes) {
-		if (this.ipBytes == null) {
-			this.ipBytes = new byte[4];
-		}
-		System.arraycopy(bytes, 0, this.ipBytes, 0, 4);
-	}
-
-	public byte[] getBytes() {
-		return this.ipBytes;
-	}
-	
-	/**
-	 * This function will be provided as UDF later.
-	 * @param addr
-	 * @return
-	 * @throws InvalidAddressException
-	 */
-	public boolean matchSubnet(String addr) throws InvalidAddressException {
-		int maskIndex;
-		if ((maskIndex=addr.indexOf('/')) != -1) {
-			IPv4 other = new IPv4(addr.substring(0, maskIndex));
-			int maskLen = Integer.valueOf(addr.substring(maskIndex+1));
-			IPv4 subnetMask = IPv4.getSubnetMask(maskLen);
-			if (this.and(subnetMask).equals(other.and(subnetMask))) {
-				return true;
-			} else {
-				return false;
-			}
-		} else {
-			throw new InvalidAddressException();
-		}
-	}
-	
-	/**
-	 * This function will be provided as UDF later.
-	 * @return
-	 */
-	public boolean matchGeoIP(/* country code */) {
-		
-		return false;
-	}
-	
-	public IPv4 and(IPv4 other) {
-		byte[] res = new byte[4];
-		byte[] obytes = other.getBytes();
-		
-		for (int i = 0; i < 4; i++) {
-			res[i] = (byte)(this.ipBytes[i] & obytes[i]);
-		}
-		
-		return new IPv4(res);
-	}
-	
-	@Override
-	public boolean equals(Object o) {
-		if (o instanceof IPv4) {
-			byte[] obytes = ((IPv4)o).getBytes();
-			for (int i = 0; i < 4; i++) {
-				if (this.ipBytes[i] != obytes[i]) {
-					return false;
-				}
-			}
-			return true;
-		}
-		return false;
-	}
-
-	/**
-	 * This is a method for range query such as 'SELECT * FROM table WHERE srcIP > 163.152.23.0 and srcIP < 163.152.23.100'
-	 */
-	@Override
-	public int compareTo(IPv4 o) {
-		byte[] obytes = o.getBytes();
-		for (int i = 0; i < 4; i++) {
-			if (this.ipBytes[i] > obytes[i]) {
-				return 1;
-			} else if (this.ipBytes[i] < obytes[i]) {
-				return -1;
-			}
-		}
-		return 0;
-	}
-	
-	@Override
-	public String toString() {
-		String str = "";
-		int i;
-		for (i = 0; i < 3; i++) {
-			str += (((int)ipBytes[i] << 24) >> 24 & 0xFF) + ".";
-		}
-		str += (((int)ipBytes[i] << 24) >> 24 & 0xFF);
-		return str;
-	}
-	
-	@Override
-	public int hashCode() {
-		return 0;
-	}
-
-	@Override
-	public void readFields(DataInput in) throws IOException {
-		in.readFully(this.ipBytes, 0, 4);
-	}
-
-	@Override
-	public void write(DataOutput out) throws IOException {
-		out.write(this.ipBytes, 0, 4);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/common/type/TimeRange.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/common/type/TimeRange.java b/tajo-common/src/main/java/tajo/common/type/TimeRange.java
deleted file mode 100644
index ad2e889..0000000
--- a/tajo-common/src/main/java/tajo/common/type/TimeRange.java
+++ /dev/null
@@ -1,80 +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 tajo.common.type;
-
-import java.nio.ByteBuffer;
-
-public class TimeRange implements Comparable<TimeRange>{
-
-	private long begin;
-	private long end;
-	
-	public TimeRange() {
-		
-	}
-	
-	public TimeRange(long begin, long end) {
-		this.set(begin, end);
-	}
-	
-	public void set(long begin, long end) {
-		this.begin = begin;
-		this.end = end;
-	}
-	
-	public long getBegin() {
-		return this.begin;
-	}
-	
-	public long getEnd() {
-		return this.end;
-	}
-
-	/*
-	 * TimeRange must not be overlapped with other TimeRange.
-	 */
-	@Override
-	public int compareTo(TimeRange o) {
-		return (int)(this.begin - o.getBegin());
-	}
-	
-	@Override
-	public boolean equals(Object o) {
-		if (o instanceof TimeRange) {
-			TimeRange tr = (TimeRange)o;
-			if (this.begin == tr.getBegin()) {
-				return true;
-			}
-		}
-		return false;
-	}
-	
-	@Override
-	public int hashCode() {
-		byte[] array = new byte[8];
-		ByteBuffer bb = ByteBuffer.wrap(array);
-		bb.putLong(this.begin);
-		return bb.hashCode();
-	}
-	
-	@Override
-	public String toString() {
-		return new String("(" + begin + ", " + end + ")");
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/conf/TajoConf.java b/tajo-common/src/main/java/tajo/conf/TajoConf.java
deleted file mode 100644
index 084a416..0000000
--- a/tajo-common/src/main/java/tajo/conf/TajoConf.java
+++ /dev/null
@@ -1,351 +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 tajo.conf;
-
-import com.google.common.collect.Maps;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-
-import java.io.PrintStream;
-import java.util.Map;
-
-public class TajoConf extends YarnConfiguration {
-  static{
-    Configuration.addDefaultResource("catalog-default.xml");
-    Configuration.addDefaultResource("catalog-site.xml");
-    Configuration.addDefaultResource("storage-default.xml");
-    Configuration.addDefaultResource("storage-site.xml");
-    Configuration.addDefaultResource("tajo-default.xml");
-    Configuration.addDefaultResource("tajo-site.xml");
-  }
-
-  private static final Map<String, ConfVars> vars = Maps.newHashMap();
-
-  public TajoConf() {
-    super();
-  }
-
-  public TajoConf(Configuration conf) {
-    super(conf);
-  }
-
-  public TajoConf(Path path) {
-    super();
-    addResource(path);
-  }
-
-  public static enum ConfVars {
-    //////////////////////////////////
-    // System Configuration
-    //////////////////////////////////
-
-    // Cluster Mode
-    CLUSTER_DISTRIBUTED("tajo.cluster.distributed", false),
-
-    // System Directories
-    ROOT_DIR("tajo.rootdir", "/tajo"),
-    TASK_LOCAL_DIR("tajo.task.localdir", ""),
-
-    // Service Addresses
-    TASKRUNNER_LISTENER_ADDRESS("tajo.master.taskrunnerlistener.addr", "0.0.0.0:0"), // used internally
-    CLIENT_SERVICE_ADDRESS("tajo.master.clientservice.addr", "0.0.0.0:9004"),
-    CLIENT_SERVICE_PORT("tajo.master.clientservice.port", 9004),
-
-    //////////////////////////////////
-    // Catalog Configuration
-    //////////////////////////////////
-    CATALOG_ADDRESS("tajo.catalog.master.addr", "127.0.0.1:9002"),
-
-    //////////////////////////////////
-    // AM and RM
-    //////////////////////////////////
-    AM_TO_RM_HEARTBEAT_INTERVAL_MS("tajo.am.scheduler.heartbeat.interval-ms", 1000),
-    AM_TO_RM_WAIT_INTERVAL_MS("tajo.am.scheduler.connection.wait.interval-ms", 360000L),
-    /** Enable blacklisting of nodes in the job.*/
-    AM_QUERY_NODE_BLACKLISTING_ENABLE("tajo.query.node-blacklisting.enable", true),
-    MAX_TASK_FAILURES_PER_TRACKER("tajo.query.maxtaskfailures.per.worker", 3),
-    AM_IGNORE_BLACKLISTING_BLACKLISTED_NODE_PERECENT("tajo.query.node-blacklisting.ignore-threshold-node-percent", 33),
-    /** how many launching TaskRunners in parallel */
-    AM_TASKRUNNER_LAUNCH_PARALLEL_NUM("tajo.master.taskrunnerlauncher.parallel.num", 16),
-
-
-
-    //////////////////////////////////
-    // Pull Server
-    //////////////////////////////////
-    PULLSERVER_PORT("tajo.pullserver.port", 0),
-    SHUFFLE_SSL_ENABLED_KEY("tajo.pullserver.ssl.enabled", false),
-
-    //////////////////////////////////
-    // Storage Configuration
-    //////////////////////////////////
-    RAWFILE_SYNC_INTERVAL("rawfile.sync.interval", null),
-    // for RCFile
-    HIVEUSEEXPLICITRCFILEHEADER("tajo.exec.rcfile.use.explicit.header", true),
-
-
-    //////////////////////////////////
-    // Physical Executors
-    //////////////////////////////////
-    EXT_SORT_BUFFER("tajo.extsort.buffer", 400000),
-    BROADCAST_JOIN_THRESHOLD("tajo.join.broadcast.threshold", (long)5 * 1048576),
-
-    //////////////////////////////////////////
-    // Distributed Query Execution Parameters
-    //////////////////////////////////////////
-    SORT_BUFFER_SIZE("tajo.sort.mb", 128),
-    JOIN_TASK_VOLUME("tajo.join.task-volume.mb", 128),
-    SORT_TASK_VOLUME("tajo.sort.task-volume.mb", 256),
-    AGGREGATION_TASK_VOLUME("tajo.task-aggregation.volume.mb", 128),
-
-    JOIN_PARTITION_VOLUME("tajo.join.part-volume.mb", 64),
-    SORT_PARTITION_VOLUME("tajo.sort.part-volume.mb", 256),
-    AGGREGATION_PARTITION_VOLUME("tajo.aggregation.part-volume.mb", 256),
-
-    //////////////////////////////////
-    // The Below is reserved
-    //////////////////////////////////
-    WORKING_DIR("tajo.query.workingdir", null),
-
-    // Query Master
-    QUERY_AM_VMEM_MB("tajo.query.am.vmem", 4096),
-    QUERY_AM_JAVA_OPT("tajo.query.am.javaopt", "-Xmx1024m"),
-
-    // QueryC
-    QUERY_NAME("tajo.query.name", "tajo query"),
-    QUEUE_NAME("tajo.query.queue.name", "default"),
-    QUERY_USERNAME("tajo.query.user.name", "anonymous"),
-    QUERY_OUTPUT_DIR_SUCCESSFUL_MARKER("tajo.query.output.successfulmaker", true),
-    QUERY_OUTPUT_DIR("tajo.query.output.dir", ""),
-    QUERY_OUTPUT_TABLE("tajo.query.output.table", ""),
-
-    // Task
-    LOCAL_TMP_DIR("tajo.task.local.tmpdir", null),
-
-    // TaskAttempt
-    APPLICATION_ATTEMPT_ID("tajo.app.attempt.id", 0),
-    TASK_ATTEMPT_ID("tajo.task.attempt.id", ""),
-    ATTEMPT_ID("tajo.query.attempt.id", ""),
-    FINAL_SUBQUERY_ID("tajo.query.finalsubquery.id", ""),
-
-    // GeoIP
-    GEOIP_DATA("tajo.geoip.data", "/usr/local/share/GeoIP/GeoIP.dat"),
-    ;
-
-    public final String varname;
-    public final String defaultVal;
-    public final int defaultIntVal;
-    public final long defaultLongVal;
-    public final float defaultFloatVal;
-    public final Class<?> valClass;
-    public final boolean defaultBoolVal;
-
-    private final VarType type;
-
-    ConfVars(String varname, String defaultVal) {
-      this.varname = varname;
-      this.valClass = String.class;
-      this.defaultVal = defaultVal;
-      this.defaultIntVal = -1;
-      this.defaultLongVal = -1;
-      this.defaultFloatVal = -1;
-      this.defaultBoolVal = false;
-      this.type = VarType.STRING;
-    }
-
-    ConfVars(String varname, int defaultIntVal) {
-      this.varname = varname;
-      this.valClass = Integer.class;
-      this.defaultVal = Integer.toString(defaultIntVal);
-      this.defaultIntVal = defaultIntVal;
-      this.defaultLongVal = -1;
-      this.defaultFloatVal = -1;
-      this.defaultBoolVal = false;
-      this.type = VarType.INT;
-    }
-
-    ConfVars(String varname, long defaultLongVal) {
-      this.varname = varname;
-      this.valClass = Long.class;
-      this.defaultVal = Long.toString(defaultLongVal);
-      this.defaultIntVal = -1;
-      this.defaultLongVal = defaultLongVal;
-      this.defaultFloatVal = -1;
-      this.defaultBoolVal = false;
-      this.type = VarType.LONG;
-    }
-
-    ConfVars(String varname, float defaultFloatVal) {
-      this.varname = varname;
-      this.valClass = Float.class;
-      this.defaultVal = Float.toString(defaultFloatVal);
-      this.defaultIntVal = -1;
-      this.defaultLongVal = -1;
-      this.defaultFloatVal = defaultFloatVal;
-      this.defaultBoolVal = false;
-      this.type = VarType.FLOAT;
-    }
-
-    ConfVars(String varname, boolean defaultBoolVal) {
-      this.varname = varname;
-      this.valClass = Boolean.class;
-      this.defaultVal = Boolean.toString(defaultBoolVal);
-      this.defaultIntVal = -1;
-      this.defaultLongVal = -1;
-      this.defaultFloatVal = -1;
-      this.defaultBoolVal = defaultBoolVal;
-      this.type = VarType.BOOLEAN;
-    }
-
-    enum VarType {
-      STRING { void checkType(String value) throws Exception { } },
-      INT { void checkType(String value) throws Exception { Integer
-          .valueOf(value); } },
-      LONG { void checkType(String value) throws Exception { Long.valueOf(value); } },
-      FLOAT { void checkType(String value) throws Exception { Float
-          .valueOf(value); } },
-      BOOLEAN { void checkType(String value) throws Exception { Boolean
-          .valueOf(value); } };
-
-      boolean isType(String value) {
-        try { checkType(value); } catch (Exception e) { return false; }
-        return true;
-      }
-      String typeString() { return name().toUpperCase();}
-      abstract void checkType(String value) throws Exception;
-    }
-  }
-
-  public static int getIntVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Integer.class);
-    return conf.getInt(var.varname, var.defaultIntVal);
-  }
-
-  public static void setIntVar(Configuration conf, ConfVars var, int val) {
-    assert (var.valClass == Integer.class);
-    conf.setInt(var.varname, val);
-  }
-
-  public int getIntVar(ConfVars var) {
-    return getIntVar(this, var);
-  }
-
-  public void setIntVar(ConfVars var, int val) {
-    setIntVar(this, var, val);
-  }
-
-  public static long getLongVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Long.class);
-    return conf.getLong(var.varname, var.defaultLongVal);
-  }
-
-  public static long getLongVar(Configuration conf, ConfVars var, long defaultVal) {
-    return conf.getLong(var.varname, defaultVal);
-  }
-
-  public static void setLongVar(Configuration conf, ConfVars var, long val) {
-    assert (var.valClass == Long.class);
-    conf.setLong(var.varname, val);
-  }
-
-  public long getLongVar(ConfVars var) {
-    return getLongVar(this, var);
-  }
-
-  public void setLongVar(ConfVars var, long val) {
-    setLongVar(this, var, val);
-  }
-
-  public static float getFloatVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Float.class);
-    return conf.getFloat(var.varname, var.defaultFloatVal);
-  }
-
-  public static float getFloatVar(Configuration conf, ConfVars var, float defaultVal) {
-    return conf.getFloat(var.varname, defaultVal);
-  }
-
-  public static void setFloatVar(Configuration conf, ConfVars var, float val) {
-    assert (var.valClass == Float.class);
-    conf.setFloat(var.varname, val);
-  }
-
-  public float getFloatVar(ConfVars var) {
-    return getFloatVar(this, var);
-  }
-
-  public void setFloatVar(ConfVars var, float val) {
-    setFloatVar(this, var, val);
-  }
-
-  public static boolean getBoolVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == Boolean.class);
-    return conf.getBoolean(var.varname, var.defaultBoolVal);
-  }
-
-  public static boolean getBoolVar(Configuration conf, ConfVars var, boolean defaultVal) {
-    return conf.getBoolean(var.varname, defaultVal);
-  }
-
-  public static void setBoolVar(Configuration conf, ConfVars var, boolean val) {
-    assert (var.valClass == Boolean.class);
-    conf.setBoolean(var.varname, val);
-  }
-
-  public boolean getBoolVar(ConfVars var) {
-    return getBoolVar(this, var);
-  }
-
-  public void setBoolVar(ConfVars var, boolean val) {
-    setBoolVar(this, var, val);
-  }
-
-  public static String getVar(Configuration conf, ConfVars var) {
-    assert (var.valClass == String.class);
-    return conf.get(var.varname, var.defaultVal);
-  }
-
-  public static String getVar(Configuration conf, ConfVars var, String defaultVal) {
-    return conf.get(var.varname, defaultVal);
-  }
-
-  public static void setVar(Configuration conf, ConfVars var, String val) {
-    assert (var.valClass == String.class);
-    conf.set(var.varname, val);
-  }
-
-  public static ConfVars getConfVars(String name) {
-    return vars.get(name);
-  }
-
-  public String getVar(ConfVars var) {
-    return getVar(this, var);
-  }
-
-  public void setVar(ConfVars var, String val) {
-    setVar(this, var, val);
-  }
-
-  public void logVars(PrintStream ps) {
-    for (ConfVars one : ConfVars.values()) {
-      ps.println(one.varname + "=" + ((get(one.varname) != null) ? get(one.varname) : ""));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/ArrayDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/ArrayDatum.java b/tajo-common/src/main/java/tajo/datum/ArrayDatum.java
deleted file mode 100644
index e1bc84c..0000000
--- a/tajo-common/src/main/java/tajo/datum/ArrayDatum.java
+++ /dev/null
@@ -1,81 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.json.GsonCreator;
-
-import static tajo.common.TajoDataTypes.Type;
-
-public class ArrayDatum extends Datum {
-  @Expose private Datum [] data;
-  public ArrayDatum(Datum [] data) {
-    super(Type.ARRAY);
-    this.data = data;
-  }
-
-  public ArrayDatum(int size) {
-    super(Type.ARRAY);
-    this.data = new Datum[size];
-  }
-
-  public Datum get(int idx) {
-    return data[idx];
-  }
-
-  public Datum [] toArray() {
-    return data;
-  }
-
-  public void put(int idx, Datum datum) {
-    data[idx] = datum;
-  }
-
-  @Override
-  public int size() {
-    return 0;
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    return 0; // TODO - to be implemented
-  }
-
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("[");
-    boolean first = true;
-    for (Datum field : data) {
-      if (first) {
-        first = false;
-      } else {
-        sb.append(",");
-      }
-      sb.append(field.asChars());
-    }
-    sb.append("]");
-
-    return sb.toString();
-  }
-
-  @Override
-  public String toJSON() {
-    return GsonCreator.getInstance().toJson(this, Datum.class);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/BitDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/BitDatum.java b/tajo-common/src/main/java/tajo/datum/BitDatum.java
deleted file mode 100644
index 5fd9ecd..0000000
--- a/tajo-common/src/main/java/tajo/datum/BitDatum.java
+++ /dev/null
@@ -1,134 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.common.TajoDataTypes;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-public class BitDatum extends Datum {
-  private static final int size = 1;
-  @Expose	byte val;
-	
-	public BitDatum() {
-		super(TajoDataTypes.Type.BIT);
-	}
-	
-	public BitDatum(byte val) {
-		this();
-		this.val = val;
-	}
-
-  public BitDatum(byte[] bytes) {
-    this(bytes[0]);
-  }
-
-  @Override
-  public char asChar() {
-    return (char)val;
-  }
-
-	@Override
-	public int asInt4() {
-		return val;
-	}
-
-  @Override
-	public long asInt8() {
-		return val;
-	}
-
-  @Override
-	public byte asByte() {
-		return val;
-	}
-
-  @Override
-	public byte[] asByteArray() {
-    byte [] bytes = new byte[1];
-    bytes[0] = this.val;
-		return bytes;
-	}
-
-  @Override
-	public float asFloat4() {
-		return val;
-	}
-
-  @Override
-	public double asFloat8() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return "0x"+val;
-	}
-	
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return val;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof BitDatum) {
-      BitDatum other = (BitDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BooleanDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case BIT:
-      return DatumFactory.createBool(this.val == (((BitDatum) datum).val));
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case BIT:
-      if (val < datum.asByte() ) {
-        return -1;
-      } else if (val > datum.asByte()) {
-        return 1;
-      } else {
-        return 0;
-      }
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/BlobDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/BlobDatum.java b/tajo-common/src/main/java/tajo/datum/BlobDatum.java
deleted file mode 100644
index 5f3ad2d..0000000
--- a/tajo-common/src/main/java/tajo/datum/BlobDatum.java
+++ /dev/null
@@ -1,163 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-
-import static tajo.common.TajoDataTypes.Type.BLOB;
-
-public class BlobDatum extends Datum {
-	@Expose private byte [] val;
-	private ByteBuffer bb = null;
-
-	public BlobDatum() {
-		super(BLOB);
-	}
-	
-	public BlobDatum(byte[] val) {
-		this();
-		this.val = val;
-		this.bb = ByteBuffer.wrap(val);	
-		bb.flip();
-	}
-	
-	public BlobDatum(ByteBuffer val) {
-		this();
-		this.val = val.array();
-		this.bb = val.duplicate();
-		bb.flip();
-	}
-	
-	public void initFromBytes() {
-		if (bb == null) {
-			bb = ByteBuffer.wrap(val);
-		}
-	}
-
-  @Override
-	public int asInt4() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getInt();
-	}
-
-  @Override
-	public long asInt8() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getLong();
-	}
-
-  @Override
-	public byte asByte() {
-		initFromBytes();
-		bb.rewind();
-		return bb.get();
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		initFromBytes();
-		bb.rewind();
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat4() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getFloat();
-	}
-
-  @Override
-	public double asFloat8() {
-		initFromBytes();
-		bb.rewind();
-		return bb.getDouble();
-	}
-
-  @Override
-	public String asChars() {
-		initFromBytes();
-		bb.rewind();
-		return new String(bb.array(), Charset.defaultCharset());
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-	  return this.val.length;
-  }
-  
-  @Override
-  public int hashCode() {
-	  initFromBytes();
-	  bb.rewind();
-    return bb.hashCode();
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof BlobDatum) {
-      BlobDatum other = (BlobDatum) obj;
-      initFromBytes();
-      other.initFromBytes();
-      return bb.equals(other.bb);
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BooleanDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case BLOB:
-    	initFromBytes();
-    	((BlobDatum)datum).initFromBytes();
-      return DatumFactory.createBool(Arrays.equals(this.val, ((BlobDatum)datum).val));
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case BLOB:
-    	initFromBytes();
-    	((BlobDatum)datum).initFromBytes();
-      return bb.compareTo(((BlobDatum) datum).bb);
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/BooleanDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/BooleanDatum.java b/tajo-common/src/main/java/tajo/datum/BooleanDatum.java
deleted file mode 100644
index f936d05..0000000
--- a/tajo-common/src/main/java/tajo/datum/BooleanDatum.java
+++ /dev/null
@@ -1,173 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.common.TajoDataTypes;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-public class BooleanDatum extends Datum {
-	@Expose private boolean val;
-
-  public BooleanDatum() {
-    super(TajoDataTypes.Type.BOOLEAN);
-  }
-
-	public BooleanDatum(boolean val) {
-		this();
-		this.val = val;
-	}
-
-  public BooleanDatum(byte byteVal) {
-    this();
-    this.val = byteVal == 1;
-  }
-
-  public BooleanDatum(int byteVal) {
-    this();
-    this.val = byteVal == 1;
-  }
-
-
-  public BooleanDatum(byte[] bytes) {
-    this(bytes[0]);
-  }
-	
-	public boolean asBool() {
-		return val;
-	}
-
-  public void setValue(boolean val) {
-    this.val = val;
-  }
-	
-	@Override
-	public short asInt2() {
-		return (short) (val ? 1 : 0);
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asInt()
-	 */
-	@Override
-	public int asInt4() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asLong()
-	 */
-	@Override
-	public long asInt8() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asByte()
-	 */
-	@Override
-	public byte asByte() {
-		return (byte) (val ? 0x01 : 0x00);
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asByteArray()
-	 */
-	@Override
-	public byte[] asByteArray() {
-	  byte [] bytes = new byte[1];
-    bytes[0] = asByte();
-	  return bytes;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asFloat()
-	 */
-	@Override
-	public float asFloat4() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asDouble()
-	 */
-	@Override
-	public double asFloat8() {
-		return val ? 1 : 0;
-	}
-
-	/* (non-Javadoc)
-	 * @see nta.common.datum.Datum#asChars()
-	 */
-	@Override
-	public String asChars() {
-		return val ? "true" : "false";
-	}
-	
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return 1;
-  }
-  
-  @Override
-  public int hashCode() {
-    return val ? 7907 : 0; // 7907 is one of the prime numbers
-  }
-  
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof BooleanDatum) {
-      BooleanDatum other = (BooleanDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-  
-  // Datum Comparator
-  public BooleanDatum equalsTo(Datum datum) {
-    switch(datum.type()) {
-      case BOOLEAN: return DatumFactory.createBool(this.val == 
-          ((BooleanDatum)datum).val);
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case BOOLEAN:
-      if (val && !datum.asBool()) {
-        return -1;
-      } else if (val && datum.asBool()) {
-        return 1;
-      } else {
-        return 0;
-      }
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/CharDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/CharDatum.java b/tajo-common/src/main/java/tajo/datum/CharDatum.java
deleted file mode 100644
index 0dc11f8..0000000
--- a/tajo-common/src/main/java/tajo/datum/CharDatum.java
+++ /dev/null
@@ -1,141 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import static tajo.common.TajoDataTypes.Type;
-
-public class CharDatum extends Datum {
-  private static final int size = 1;
-  @Expose char val;
-
-	public CharDatum() {
-		super(Type.CHAR);
-	}
-
-	public CharDatum(byte val) {
-		this();
-		this.val = (char)val;
-	}
-
-  public CharDatum(byte [] bytes) {
-    this(bytes[0]);
-  }
-
-	public CharDatum(char val) {
-	  this();
-	  this.val = val;
-	}
-
-  @Override
-  public char asChar() {
-    return val;
-  }
-
-  @Override
-	public int asInt4() {
-		return val;
-	}
-
-  @Override
-	public long asInt8() {
-		return val;
-	}
-
-  @Override
-	public byte asByte() {
-		return (byte)val;
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		byte [] bytes = new byte[1];
-    bytes[0] = (byte) val;
-		return bytes;
-	}
-
-  @Override
-	public float asFloat4() {
-		return val;
-	}
-
-  @Override
-	public double asFloat8() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return String.valueOf(val);
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return val;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof CharDatum) {
-      CharDatum other = (CharDatum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BooleanDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case CHAR:
-      return DatumFactory.createBool(this.val == (((CharDatum) datum).val));
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case CHAR:
-      if (val < datum.asChar()) {
-        return -1;
-      } else if (val > datum.asChar()) {
-        return 1;
-      } else {
-        return 0;
-      }
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/Datum.java b/tajo-common/src/main/java/tajo/datum/Datum.java
deleted file mode 100644
index f4772ef..0000000
--- a/tajo-common/src/main/java/tajo/datum/Datum.java
+++ /dev/null
@@ -1,158 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidCastException;
-import tajo.datum.exception.InvalidOperationException;
-
-import static tajo.common.TajoDataTypes.Type;
-
-public abstract class Datum implements Comparable<Datum> {
-	@Expose
-	private Type type;
-	
-	@SuppressWarnings("unused")
-  private Datum() {
-	}
-	
-	public Datum(Type type) {
-		this.type = type;
-	}
-	
-	public Type type() {
-		return this.type;
-	}
-
-  public boolean isNull() {
-    return false;
-  }
-	
-	public boolean asBool() {
-    throw new InvalidCastException(type + " cannot be casted to BOOL type");
-  }
-
-  public byte asByte() {
-    throw new InvalidCastException(type + " cannot be casted to BYTE type");
-  }
-
-  public char asChar() {
-    throw new InvalidCastException(type + " cannot be casted to CHAR type");
-  }
-
-	public short asInt2() {
-    throw new InvalidCastException(type + " cannot be casted to SHORT type");
-  }
-	public int asInt4() {
-    throw new InvalidCastException(type + " cannot be casted to INT type");
-  }
-
-  public long asInt8() {
-    throw new InvalidCastException(type + " cannot be casted to LONG type");
-  }
-
-	public byte [] asByteArray() {
-    throw new InvalidCastException(type + " cannot be casted to BYTES type");
-  }
-
-	public float asFloat4() {
-    throw new InvalidCastException(type + " cannot be casted to FLOAT type");
-  }
-
-	public double asFloat8() {
-    throw new InvalidCastException(type + " cannot be casted to DOUBLE type");
-  }
-
-	public String asChars() {
-    throw new InvalidCastException(type + " cannot be casted to STRING type");
-  }
-	
-	public boolean isNumeric() {
-	  return isNumber() || isReal();
-	}
-	
-	public boolean isNumber() {
-	  return 
-	      this.type == Type.INT2 ||
-	      this.type == Type.INT4 ||
-	      this.type == Type.INT8;
-	}
-	
-	public boolean isReal() {
-    return 
-        this.type == Type.FLOAT4||
-        this.type == Type.FLOAT8;
-  }
-	
-	public abstract int size();
-	
-	public Datum plus(Datum datum) {
-	  throw new InvalidOperationException(datum.type);
-	}
-	
-	public Datum minus(Datum datum) {
-	  throw new InvalidOperationException(datum.type);
-	}
-	
-	public Datum multiply(Datum datum) {
-	  throw new InvalidOperationException(datum.type);
-	}
-	
-	public Datum divide(Datum datum) {
-	  throw new InvalidOperationException(datum.type);
-	}
-
-  public Datum modular(Datum datum) {
-    throw new InvalidOperationException(datum.type);
-  }
-	
-	public BooleanDatum equalsTo(Datum datum) {
-    if (this instanceof NullDatum || datum instanceof NullDatum) {
-    // TODO - comparing any value against null will be always unknown
-      return DatumFactory.createBool(false);
-    } else {
-	    return DatumFactory.createBool(compareTo(datum) == 0);
-    }
-	}
-
-	public BooleanDatum lessThan(Datum datum) {
-    return DatumFactory.createBool(compareTo(datum) < 0);
-	}
-	
-	public BooleanDatum lessThanEqual(Datum datum) {
-    return DatumFactory.createBool(compareTo(datum) <= 0);
-	}	
-	
-	public BooleanDatum greaterThan(Datum datum) {
-    return DatumFactory.createBool(compareTo(datum) > 0);
-	}
-	
-	public BooleanDatum greaterThanEqual(Datum datum) {
-    return DatumFactory.createBool(compareTo(datum) >= 0);
-	}
-	
-  public abstract int compareTo(Datum datum);
-
-  public abstract String toJSON();
-
-  @Override
-  public String toString() {
-    return asChars();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/DatumFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/tajo/datum/DatumFactory.java
deleted file mode 100644
index 2e1cd79..0000000
--- a/tajo-common/src/main/java/tajo/datum/DatumFactory.java
+++ /dev/null
@@ -1,130 +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 tajo.datum;
-
-public class DatumFactory {
-  public static NullDatum createNullDatum() {
-    return NullDatum.get();
-  }
-  
-  public static BooleanDatum createBool(String val) {
-    boolean boolVal = val.equalsIgnoreCase("true");
-    return new BooleanDatum(boolVal);
-  }
-  
-  public static BooleanDatum createBool(byte val) {
-    boolean boolVal = val == 0x01;
-    return new BooleanDatum(boolVal);
-  }
-  
-  public static BooleanDatum createBool(boolean val) {
-    return new BooleanDatum(val);
-  }
-  /*
-  public static BoolDatum createBool(int val) {
-    return new BoolDatum(val);
-  }
-  */
-  
-	public static BitDatum createBit(byte val) {
-		return new BitDatum(val);
-	}
-  /*
-  public static ByteDatum createBit(int val) {
-    return new ByteDatum(val);
-  }*/
-
-  public static CharDatum createChar(char val) {
-    return new CharDatum(val);
-  }
-
-  public static CharDatum createChar(byte val) {
-    return new CharDatum(val);
-  }
-
-  /*
-  public static CharDatum createChar(Integer val) {
-    return new CharDatum(val);
-  }
-  */
-	
-	public static Int2Datum createInt2(short val) {
-		return new Int2Datum(val);
-	}
-	
-	public static Int2Datum createInt2(String val) {
-	  return new Int2Datum(Short.valueOf(val));
-	}
-	
-	public static Int4Datum createInt4(int val) {
-		return new Int4Datum(val);
-	}
-	
-	public static Int4Datum createInt4(String val) {
-	  return new Int4Datum(Integer.valueOf(val));
-	}
-	
-	public static Int8Datum createInt8(long val) {
-		return new Int8Datum(val);
-	}
-	
-	public static Int8Datum createInt8(String val) {
-	  return new Int8Datum(Long.valueOf(val));
-	}
-	
-	public static Float4Datum createFloat4(float val) {
-		return new Float4Datum(val);
-	}
-	
-	public static Float4Datum createFloat4(String val) {
-	  return new Float4Datum(Float.valueOf(val));
-	}
-	
-	public static Float8Datum createFloat8(double val) {
-		return new Float8Datum(val);
-	}
-	
-	public static Float8Datum createFloat8(String val) {
-	  return new Float8Datum(Double.valueOf(val));
-	}
-	
-  public static TextDatum createText(String val) {
-    return new TextDatum(val);
-  }
-
-  public static TextDatum createText(byte[] val) {
-    return new TextDatum(val);
-  }
-	
-	public static BlobDatum createBlob(byte[] val) {
-    return new BlobDatum(val);
-  }
-	
-	public static BlobDatum createBlob(String val) {
-	  return new BlobDatum(val.getBytes());
-	}
-	
-	public static Inet4Datum createInet4(byte[] val) {
-	  return new Inet4Datum(val);
-	}
-	
-	public static Inet4Datum createInet4(String val) {
-	  return new Inet4Datum(val);
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/Float4Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/Float4Datum.java b/tajo-common/src/main/java/tajo/datum/Float4Datum.java
deleted file mode 100644
index f3d2eb4..0000000
--- a/tajo-common/src/main/java/tajo/datum/Float4Datum.java
+++ /dev/null
@@ -1,279 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.common.TajoDataTypes;
-import tajo.datum.exception.InvalidCastException;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-
-public class Float4Datum extends NumericDatum {
-  private static final int size = 4;
-  @Expose float val;
-
-	public Float4Datum() {
-		super(TajoDataTypes.Type.FLOAT4);
-	}
-	
-	public Float4Datum(float val) {
-		this();
-		this.val = val;
-	}
-
-  public Float4Datum(byte[] bytes) {
-    this();
-    ByteBuffer bb = ByteBuffer.wrap(bytes);
-    this.val = bb.getFloat();
-  }
-	
-	public boolean asBool() {
-		throw new InvalidCastException();
-	}
-	
-	@Override
-	public short asInt2() {
-		return (short) val;
-	}
-
-  @Override
-	public int asInt4() {
-		return (int) val;
-	}
-
-  @Override
-	public long asInt8() {
-		return (long) val;
-	}
-
-  @Override
-	public byte asByte() {
-		throw new InvalidCastException();
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		ByteBuffer bb = ByteBuffer.allocate(4);
-		bb.putFloat(val);
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat4() {
-		return val;
-	}
-
-  @Override
-	public double asFloat8() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return ""+val;
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return (int) val;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof Float4Datum) {
-      Float4Datum other = (Float4Datum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BooleanDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createBool(val == datum.asInt2());
-    case INT4:
-      return DatumFactory.createBool(val == datum.asInt4());
-    case INT8:
-      return DatumFactory.createBool(val == datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createBool(val == datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createBool(val == datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case INT2:
-        if (val < datum.asInt2()) {
-          return -1;
-        } else if (datum.asInt2() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT4:
-        if (val < datum.asInt4()) {
-          return -1;
-        } else if (datum.asInt4() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT8:
-        if (val < datum.asInt8()) {
-          return -1;
-        } else if (datum.asInt8() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT4:
-        if (val < datum.asFloat4()) {
-          return -1;
-        } else if (datum.asFloat4() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT8:
-        if (val < datum.asFloat8()) {
-          return -1;
-        } else if (datum.asFloat8() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum plus(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat4(val + datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat4(val + datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val + datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val + datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val + datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum minus(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat4(val - datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat4(val - datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val - datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val - datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val - datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum multiply(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat4(val * datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat4(val * datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val * datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val * datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val * datum.asFloat8());
-    default:
-      throw new InvalidOperationException();
-    }
-  }
-
-  @Override
-  public Datum divide(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat4(val / datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat4(val / datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val / datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val / datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val / datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum modular(Datum datum) {
-    switch (datum.type()) {
-      case INT2:
-        return DatumFactory.createFloat4(val / datum.asInt2());
-      case INT4:
-        return DatumFactory.createFloat4(val / datum.asInt4());
-      case INT8:
-        return DatumFactory.createFloat4(val / datum.asInt8());
-      case FLOAT4:
-        return DatumFactory.createFloat4(val / datum.asFloat4());
-      case FLOAT8:
-        return DatumFactory.createFloat8(val / datum.asFloat8());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public void inverseSign() {
-    this.val = - val;    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/Float8Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/Float8Datum.java b/tajo-common/src/main/java/tajo/datum/Float8Datum.java
deleted file mode 100644
index 67a0fe8..0000000
--- a/tajo-common/src/main/java/tajo/datum/Float8Datum.java
+++ /dev/null
@@ -1,271 +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 tajo.datum;
-
-import com.google.gson.annotations.Expose;
-import tajo.common.TajoDataTypes;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import java.nio.ByteBuffer;
-
-public class Float8Datum extends NumericDatum {
-  private static final int size = 8;
-  @Expose private double val;
-
-	public Float8Datum() {
-		super(TajoDataTypes.Type.FLOAT8);
-	}
-	
-	public Float8Datum(double val) {
-		this();
-		this.val = val;
-	}
-
-  public Float8Datum(byte[] bytes) {
-    this();
-    ByteBuffer bb = ByteBuffer.wrap(bytes);
-    this.val = bb.getDouble();
-  }
-	
-	@Override
-	public short asInt2() {
-		return (short) val;
-	}
-
-	@Override
-	public int asInt4() {
-		return (int) val;
-	}
-
-  @Override
-	public long asInt8() {
-		return (long) val;
-	}
-
-  @Override
-	public byte[] asByteArray() {
-		ByteBuffer bb = ByteBuffer.allocate(8);
-		bb.putDouble(val);
-		return bb.array();
-	}
-
-  @Override
-	public float asFloat4() {
-		return (float) val;
-	}
-
-  @Override
-	public double asFloat8() {
-		return val;
-	}
-
-  @Override
-	public String asChars() {
-		return ""+val;
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return (int) val;
-  }
-  
-  public boolean equals(Object obj) {
-    if (obj instanceof Float8Datum) {
-      Float8Datum other = (Float8Datum) obj;
-      return val == other.val;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BooleanDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createBool(val == datum.asInt2());
-    case INT4:
-      return DatumFactory.createBool(val == datum.asInt4());
-    case INT8:
-      return DatumFactory.createBool(val == datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createBool(val == datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createBool(val == datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-      case INT2:
-        if (val < datum.asInt2()) {
-          return -1;
-        } else if (datum.asInt2() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT4:
-        if (val < datum.asInt4()) {
-          return -1;
-        } else if (datum.asInt4() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case INT8:
-        if (val < datum.asInt8()) {
-          return -1;
-        } else if (datum.asInt8() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT4:
-        if (val < datum.asFloat4()) {
-          return -1;
-        } else if (datum.asFloat4() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      case FLOAT8:
-        if (val < datum.asFloat8()) {
-          return -1;
-        } else if (datum.asFloat8() < val) {
-          return 1;
-        } else {
-          return 0;
-        }
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum plus(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat8(val + datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat8(val + datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val + datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val + datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val + datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum minus(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat8(val - datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat8(val - datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val - datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val - datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val - datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum multiply(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat8(val * datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat8(val * datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val * datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val * datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val * datum.asFloat8());
-    default:
-      throw new InvalidOperationException();
-    }
-  }
-
-  @Override
-  public Datum divide(Datum datum) {
-    switch (datum.type()) {
-    case INT2:
-      return DatumFactory.createFloat8(val / datum.asInt2());
-    case INT4:
-      return DatumFactory.createFloat8(val / datum.asInt4());
-    case INT8:
-      return DatumFactory.createFloat8(val / datum.asInt8());
-    case FLOAT4:
-      return DatumFactory.createFloat8(val / datum.asFloat4());
-    case FLOAT8:
-      return DatumFactory.createFloat8(val / datum.asFloat8());
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-
-  @Override
-  public Datum modular(Datum datum) {
-    switch (datum.type()) {
-      case INT2:
-        return DatumFactory.createFloat8(val % datum.asInt2());
-      case INT4:
-        return DatumFactory.createFloat8(val % datum.asInt4());
-      case INT8:
-        return DatumFactory.createFloat8(val % datum.asInt8());
-      case FLOAT4:
-        return DatumFactory.createFloat8(val % datum.asFloat4());
-      case FLOAT8:
-        return DatumFactory.createFloat8(val % datum.asFloat8());
-      default:
-        throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  @Override
-  public void inverseSign() {   
-    this.val = -val;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/bc6359b8/tajo-common/src/main/java/tajo/datum/Inet4Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/tajo/datum/Inet4Datum.java b/tajo-common/src/main/java/tajo/datum/Inet4Datum.java
deleted file mode 100644
index ce43494..0000000
--- a/tajo-common/src/main/java/tajo/datum/Inet4Datum.java
+++ /dev/null
@@ -1,138 +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 tajo.datum;
-
-import com.google.common.base.Preconditions;
-import com.google.gson.annotations.Expose;
-import tajo.datum.exception.InvalidOperationException;
-import tajo.datum.json.GsonCreator;
-
-import static tajo.common.TajoDataTypes.Type;
-
-public class Inet4Datum extends Datum {
-  private static final int size = 4;
-  @Expose private int address;
-
-	public Inet4Datum() {
-		super(Type.INET4);
-	}
-	
-	public Inet4Datum(String addr) {
-		this();
-		String [] elems = addr.split("\\.");
-		address  = Integer.valueOf(elems[3]) & 0xFF;
-    address |= ((Integer.valueOf(elems[2]) << 8) & 0xFF00);
-    address |= ((Integer.valueOf(elems[1]) << 16) & 0xFF0000);
-    address |= ((Integer.valueOf(elems[0]) << 24) & 0xFF000000);
-	}
-	
-	public Inet4Datum(byte[] addr) {
-		this();
-		Preconditions.checkArgument(addr.length == size);
-		address  = addr[3] & 0xFF;
-    address |= ((addr[2] << 8) & 0xFF00);
-    address |= ((addr[1] << 16) & 0xFF0000);
-    address |= ((addr[0] << 24) & 0xFF000000);
-	}
-
-	@Override
-	public int asInt4() {
-		return this.address;
-	}
-
-	@Override
-	public long asInt8() {
-	  return this.address;
-	}
-
-	@Override
-	public byte[] asByteArray() {
-	  byte[] addr = new byte[size];
-	  addr[0] = (byte) ((address >>> 24) & 0xFF);
-	  addr[1] = (byte) ((address >>> 16) & 0xFF);
-	  addr[2] = (byte) ((address >>> 8) & 0xFF);
-	  addr[3] = (byte) (address & 0xFF);
-	  return addr;
-	}
-
-	@Override
-	public String asChars() {
-		return numericToTextFormat(asByteArray());
-	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
-  public int size() {
-    return size;
-  }
-  
-  @Override
-  public int hashCode() {
-    return address;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof Inet4Datum) {
-      Inet4Datum other = (Inet4Datum) obj;
-      return this.address == other.address;
-    }
-    
-    return false;
-  }
-
-  @Override
-  public BooleanDatum equalsTo(Datum datum) {
-    switch (datum.type()) {
-    case INET4:
-    	return DatumFactory.createBool(this.address == ((Inet4Datum)datum).address);
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  @Override
-  public int compareTo(Datum datum) {
-    switch (datum.type()) {
-    case INET4:
-      byte [] bytes = asByteArray();
-      byte [] other = datum.asByteArray();
-      
-      for (int i = 0; i < 4; i++) {
-        if (bytes[i] > other[i]) {
-          return 1;
-        } else if (bytes[i] < other[i]) {
-          return -1;
-        }
-      }
-      
-    default:
-      throw new InvalidOperationException(datum.type());
-    }
-  }
-  
-  static String numericToTextFormat(byte[] src) {
-    return (src[0] & 0xff) + "." + (src[1] & 0xff) + "." + (src[2] & 0xff)
-        + "." + (src[3] & 0xff);
-  }
-}