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/08/08 04:17:14 UTC

[3/4] TAJO-95: Eliminate the lazy copy approach from the classes wrapping protobuf-generated classes. (hyunsik)

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStat.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStat.java
index ccfae2b..f7793c3 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStat.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/ColumnStat.java
@@ -22,140 +22,92 @@
 package org.apache.tajo.catalog.statistics;
 
 import com.google.common.base.Objects;
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
 import com.google.gson.annotations.Expose;
 import com.google.protobuf.ByteString;
+import org.apache.tajo.json.GsonObject;
 import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.ColumnStatProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.ColumnStatProtoOrBuilder;
 import org.apache.tajo.common.ProtoObject;
 import org.apache.tajo.datum.Datum;
 import org.apache.tajo.util.TUtil;
 
-public class ColumnStat implements ProtoObject<ColumnStatProto>, Cloneable {
-  private ColumnStatProto proto = ColumnStatProto.getDefaultInstance();
-  private ColumnStatProto.Builder builder = null;
-  private boolean viaProto = false;
+public class ColumnStat implements ProtoObject<ColumnStatProto>, Cloneable, GsonObject {
+  private ColumnStatProto.Builder builder = ColumnStatProto.newBuilder();
 
-  @Expose private Column column = null;
-  @Expose private Long numDistVals = null;
-  @Expose private Long numNulls = null;
-  @Expose private Datum minValue = null;
-  @Expose private Datum maxValue = null;
+  @Expose private Column column = null; // required
+  @Expose private Long numDistVals = null; // optional
+  @Expose private Long numNulls = null; // optional
+  @Expose private Datum minValue = null; // optional
+  @Expose private Datum maxValue = null; // optional
 
   public ColumnStat(Column column) {
-    builder = ColumnStatProto.newBuilder();
     this.column = column;
     numDistVals = 0l;
     numNulls = 0l;
   }
 
   public ColumnStat(ColumnStatProto proto) {
-    this.proto = proto;
-    this.viaProto = true;
-  }
+    this.column = new Column(proto.getColumn());
 
-  public Column getColumn() {
-    ColumnStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (column != null) {
-      return column;
+    if (proto.hasNumDistVal()) {
+      this.numDistVals = proto.getNumDistVal();
+    }
+    if (proto.hasNumNulls()) {
+      this.numNulls = proto.getNumNulls();
+    }
+    if (proto.hasMinValue()) {
+      this.minValue = TupleUtil.createFromBytes(getColumn().getDataType(), proto.getMinValue().toByteArray());
     }
-    if (!p.hasColumn()) {
-      return null;
+    if (proto.hasMaxValue()) {
+      this.maxValue = TupleUtil.createFromBytes(getColumn().getDataType(), proto.getMaxValue().toByteArray());
     }
-    this.column = new Column(p.getColumn());
+  }
 
+  public Column getColumn() {
     return this.column;
   }
 
   public Long getNumDistValues() {
-    ColumnStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (numDistVals != null) {
-      return this.numDistVals;
-    }
-    if (!p.hasNumDistVal()) {
-      return null;
-    }
-    this.numDistVals = p.getNumDistVal();
-
     return this.numDistVals;
   }
 
   public void setNumDistVals(long numDistVals) {
-    setModified();
     this.numDistVals = numDistVals;
   }
 
   public boolean minIsNotSet() {
-    return minValue == null && (!proto.hasMinValue());
+    return minValue == null;
   }
 
   public Datum getMinValue() {
-    ColumnStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (minValue != null) {
-      return this.minValue;
-    }
-    if (!p.hasMinValue()) {
-      return null;
-    }
-    this.minValue = TupleUtil.createFromBytes(getColumn().getDataType(), p.getMinValue().toByteArray());
-
     return this.minValue;
   }
 
   public void setMinValue(Datum minValue) {
-    setModified();
     this.minValue = minValue;
   }
 
   public boolean maxIsNotSet() {
-    return maxValue == null && (!proto.hasMaxValue());
+    return maxValue == null;
   }
 
   public Datum getMaxValue() {
-    ColumnStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (maxValue != null) {
-      return this.maxValue;
-    }
-    if (!p.hasMaxValue()) {
-      return null;
-    }
-    this.maxValue = TupleUtil.createFromBytes(getColumn().getDataType(), p.getMaxValue().toByteArray());
-
     return this.maxValue;
   }
 
   public void setMaxValue(Datum maxValue) {
-    setModified();
     this.maxValue = maxValue;
   }
 
   public Long getNumNulls() {
-    ColumnStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (numNulls != null) {
-      return this.numNulls;
-    }
-    if (!p.hasNumNulls()) {
-      return null;
-    }
-    this.numNulls = p.getNumNulls();
-
     return this.numNulls;
   }
 
   public void setNumNulls(long numNulls) {
-    setModified();
     this.numNulls = numNulls;
   }
 
-  private void setModified() {
-    if (viaProto && builder == null) {
-      builder = ColumnStatProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
   public boolean equals(Object obj) {
     if (obj instanceof ColumnStat) {
       ColumnStat other = (ColumnStat) obj;
@@ -175,7 +127,7 @@ public class ColumnStat implements ProtoObject<ColumnStatProto>, Cloneable {
 
   public Object clone() throws CloneNotSupportedException {
     ColumnStat stat = (ColumnStat) super.clone();
-    initFromProto();
+    stat.builder = ColumnStatProto.newBuilder();
     stat.column = (Column) this.column.clone();
     stat.numDistVals = numDistVals;
     stat.numNulls = numNulls;
@@ -186,46 +138,19 @@ public class ColumnStat implements ProtoObject<ColumnStatProto>, Cloneable {
   }
 
   public String toString() {
-    initFromProto();
-    Gson gson = new GsonBuilder().setPrettyPrinting().
-        excludeFieldsWithoutExposeAnnotation().create();
-    return gson.toJson(this);
+    return CatalogGsonHelper.getPrettyInstance().toJson(this, ColumnStat.class);
   }
 
   @Override
-  public void initFromProto() {
-    ColumnStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.column == null && p.hasColumn()) {
-      this.column = new Column(p.getColumn());
-    }
-    if (this.numDistVals == null && p.hasNumDistVal()) {
-      this.numDistVals = p.getNumDistVal();
-    }
-    if (this.numNulls == null && p.hasNumNulls()) {
-      this.numNulls = p.getNumNulls();
-    }
-    if (this.minValue == null && p.hasMinValue()) {
-      this.minValue = TupleUtil.createFromBytes(getColumn().getDataType(), p.getMinValue().toByteArray());
-    }
-    if (this.maxValue == null && p.hasMaxValue()) {
-      this.maxValue = TupleUtil.createFromBytes(getColumn().getDataType(), p.getMaxValue().toByteArray());
-    }
+  public String toJson() {
+    return CatalogGsonHelper.toJson(this, ColumnStat.class);
   }
 
+
   @Override
   public ColumnStatProto getProto() {
-    if (!viaProto) {
-      mergeLocalToBuilder();
-      proto = builder.build();
-      viaProto = true;
-    }
-
-    return proto;
-  }
-
-  private void mergeLocalToBuilder() {
     if (builder == null) {
-      builder = ColumnStatProto.newBuilder(proto);
+      builder = ColumnStatProto.newBuilder();
     }
     if (this.column != null) {
       builder.setColumn(this.column.getProto());
@@ -242,5 +167,7 @@ public class ColumnStat implements ProtoObject<ColumnStatProto>, Cloneable {
     if (this.maxValue != null) {
       builder.setMaxValue(ByteString.copyFrom(this.maxValue.asByteArray()));
     }
+
+    return builder.build();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatSet.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatSet.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatSet.java
index d3a3c3e..f8d24b0 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatSet.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatSet.java
@@ -20,7 +20,7 @@ package org.apache.tajo.catalog.statistics;
 
 import com.google.common.collect.Maps;
 import com.google.gson.annotations.Expose;
-import org.apache.tajo.catalog.json.GsonCreator;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.StatProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.StatSetProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.StatSetProtoOrBuilder;
@@ -83,14 +83,7 @@ public class StatSet implements ProtoObject<StatSetProto>, Cloneable {
   }
   
   public Object clone() throws CloneNotSupportedException {
-    StatSet group = (StatSet) super.clone();
-    initFromProto();
-    group.stats = Maps.newHashMap();
-    for (Stat stat : stats.values()) {
-      group.stats.put(stat.getType(), (Stat) stat.clone());
-    }
-    
-    return group;
+    return new StatSet(this.getProto());
   }
 
   private void initStats() {
@@ -112,17 +105,6 @@ public class StatSet implements ProtoObject<StatSetProto>, Cloneable {
   }
 
   @Override
-  public void initFromProto() {
-    StatSetProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.stats == null && p.getStatsCount() > 0) {
-      this.stats = Maps.newHashMap();
-      for (StatProto statProto : p.getStatsList()) {
-        this.stats.put(statProto.getType(), new Stat(statProto));
-      }
-    }
-  }
-
-  @Override
   public StatSetProto getProto() {
     if (!viaProto) {
       mergeLocalToBuilder();
@@ -146,6 +128,6 @@ public class StatSet implements ProtoObject<StatSetProto>, Cloneable {
   }
   
   public String toString() {
-    return GsonCreator.getPrettyInstance().toJson(this);
+    return CatalogGsonHelper.getPrettyInstance().toJson(this);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStat.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStat.java
index 56396b1..950e9b5 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStat.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/TableStat.java
@@ -23,158 +23,118 @@ package org.apache.tajo.catalog.statistics;
 
 import com.google.common.base.Objects;
 import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.ColumnStatProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableStatProto;
-import org.apache.tajo.catalog.proto.CatalogProtos.TableStatProtoOrBuilder;
 import org.apache.tajo.common.ProtoObject;
+import org.apache.tajo.util.TUtil;
 
 import java.util.ArrayList;
 import java.util.List;
 
-public class TableStat implements ProtoObject<TableStatProto>, Cloneable {
-  private TableStatProto proto = TableStatProto.getDefaultInstance();
-  private TableStatProto.Builder builder = null;
-  private boolean viaProto = false;
+public class TableStat implements ProtoObject<TableStatProto>, Cloneable, GsonObject {
+  private TableStatProto.Builder builder = TableStatProto.newBuilder();
 
-  @Expose private Long numRows = null;
-  @Expose private Long numBytes = null;
-  @Expose private Integer numBlocks = null;
-  @Expose private Integer numPartitions = null;
-  @Expose private Long avgRows = null;
-  @Expose private List<ColumnStat> columnStats = null;
+  @Expose private Long numRows = null; // required
+  @Expose private Long numBytes = null; // required
+  @Expose private Integer numBlocks = null; // optional
+  @Expose private Integer numPartitions = null; // optional
+  @Expose private Long avgRows = null; // optional
+  @Expose private List<ColumnStat> columnStats = null; // repeated
 
   public TableStat() {
-    builder = TableStatProto.newBuilder();
     numRows = 0l;
     numBytes = 0l;
     numBlocks = 0;
     numPartitions = 0;
     avgRows = 0l;
+    columnStats = TUtil.newList();
   }
 
   public TableStat(TableStatProto proto) {
-    this.proto = proto;
-    this.viaProto = true;
-  }
+    this.numRows = proto.getNumRows();
+    this.numBytes = proto.getNumBytes();
 
-  public Long getNumRows() {
-    TableStatProtoOrBuilder p = viaProto ? proto : builder;
-    if(numRows != null) {
-      return this.numRows;
+    if (proto.hasNumBlocks()) {
+      this.numBlocks = proto.getNumBlocks();
+    }
+    if (proto.hasNumPartitions()) {
+      this.numPartitions = proto.getNumPartitions();
     }
-    if(!p.hasNumRows()) {
-      return 0l;
+    if (proto.hasAvgRows()) {
+      this.avgRows = proto.getAvgRows();
     }
-    this.numRows = p.getNumRows();
 
+    this.columnStats = TUtil.newList();
+    for (ColumnStatProto colProto : proto.getColStatList()) {
+      columnStats.add(new ColumnStat(colProto));
+    }
+  }
+
+  public Long getNumRows() {
     return this.numRows;
   }
 
   public void setNumRows(long numRows) {
-    setModified();
     this.numRows = numRows;
   }
 
   public Integer getNumBlocks() {
-    TableStatProtoOrBuilder p = viaProto ? proto : builder;
-    if(numBlocks != null) {
-      return this.numBlocks;
-    }
-    if(!p.hasNumBlocks()) {
-      return 0;
-    }
-    this.numBlocks = p.getNumBlocks();
-
     return this.numBlocks;
   }
 
   public void setNumBytes(long numBytes) {
-    setModified();
     this.numBytes = numBytes;
   }
 
   public Long getNumBytes() {
-    TableStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (numBytes != null) {
-      return this.numBytes;
-    }
-    if (!p.hasNumBytes()) {
-      return 0l;
-    }
-    this.numBytes = p.getNumBytes();
     return this.numBytes;
   }
 
   public void setNumBlocks(int numBlocks) {
-    setModified();
     this.numBlocks = numBlocks;
   }
 
   public Integer getNumPartitions() {
-    TableStatProtoOrBuilder p = viaProto ? proto : builder;
-    if(numPartitions != null) {
-      return this.numPartitions;
-    }
-    if(!p.hasNumPartitions()) {
-      return 0;
-    }
-    this.numPartitions = p.getNumPartitions();
-
     return this.numPartitions;
   }
 
   public void setNumPartitions(int numPartitions) {
-    setModified();
     this.numPartitions = numPartitions;
   }
 
   public Long getAvgRows() {
-    TableStatProtoOrBuilder p = viaProto ? proto : builder;
-    if(avgRows != null) {
-      return this.avgRows;
-    }
-    if(!p.hasAvgRows()) {
-      return 0l;
-    }
-    this.avgRows = p.getAvgRows();
-
     return this.avgRows;
   }
 
   public void setAvgRows(long avgRows) {
-    setModified();
     this.avgRows = avgRows;
   }
 
   public List<ColumnStat> getColumnStats() {
-    initColumnStats();
     return this.columnStats;
   }
 
   public void setColumnStats(List<ColumnStat> columnStats) {
-    setModified();
     this.columnStats = new ArrayList<ColumnStat>(columnStats);
   }
 
   public void addColumnStat(ColumnStat columnStat) {
-    initColumnStats();
     this.columnStats.add(columnStat);
   }
 
   public boolean equals(Object obj) {
     if (obj instanceof TableStat) {
       TableStat other = (TableStat) obj;
-      initFromProto();
-      other.initFromProto();
 
       return this.numRows.equals(other.numRows)
           && this.numBytes.equals(other.numBytes)
-          && this.numBlocks.equals(other.numBlocks)
-          && this.numPartitions.equals(other.numPartitions)
-          && this.avgRows.equals(other.avgRows)
-          && columnStats.equals(other.columnStats);
+          && TUtil.checkEquals(this.numBlocks, other.numBlocks)
+          && TUtil.checkEquals(this.numPartitions, other.numPartitions)
+          && TUtil.checkEquals(this.avgRows, other.avgRows)
+          && TUtil.checkEquals(this.columnStats, other.columnStats);
     } else {
       return false;
     }
@@ -187,7 +147,7 @@ public class TableStat implements ProtoObject<TableStatProto>, Cloneable {
 
   public Object clone() throws CloneNotSupportedException {
     TableStat stat = (TableStat) super.clone();
-    initFromProto();
+    stat.builder = TableStatProto.newBuilder();
     stat.numRows = numRows;
     stat.numBytes = numBytes;
     stat.numBlocks = numBlocks;
@@ -198,75 +158,26 @@ public class TableStat implements ProtoObject<TableStatProto>, Cloneable {
   }
 
   public String toString() {
-    Gson gson = new GsonBuilder().setPrettyPrinting().
-        excludeFieldsWithoutExposeAnnotation().create();
+    Gson gson = CatalogGsonHelper.getPrettyInstance();
     return gson.toJson(this);
   }
 
-  private void initColumnStats() {
-    if (this.columnStats != null) {
-      return;
-    }
-    TableStatProtoOrBuilder p = viaProto ? proto : builder;
-    this.columnStats = new ArrayList<ColumnStat>();
-    for (ColumnStatProto colProto : p.getColStatList()) {
-      columnStats.add(new ColumnStat(colProto));
-    }
-  }
-
-  private void setModified() {
-    if (viaProto && builder == null) {
-      builder = TableStatProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-
   @Override
-  public void initFromProto() {
-    TableStatProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.numRows == null && p.hasNumRows()) {
-      this.numRows = p.getNumRows();
-    }
-    if (this.numBytes == null && p.hasNumBytes()) {
-      this.numBytes = p.getNumBytes();
-    }
-    if (this.numBlocks == null && p.hasNumBlocks()) {
-      this.numBlocks = p.getNumBlocks();
-    }
-    if (this.numPartitions == null && p.hasNumPartitions()) {
-      this.numPartitions = p.getNumPartitions();
-    }
-    if (this.avgRows == null && p.hasAvgRows()) {
-      this.avgRows = p.getAvgRows();
-    }
-
-    initColumnStats();
-    for (ColumnStat col : columnStats) {
-      col.initFromProto();
-    }
+  public String toJson() {
+    return CatalogGsonHelper.toJson(this, TableStat.class);
   }
 
   @Override
   public TableStatProto getProto() {
-    if (!viaProto) {
-      mergeLocalToBuilder();
-      proto = builder.build();
-      viaProto = true;
+    if (builder == null) {
+      builder = TableStatProto.newBuilder();
+    } else {
+      builder.clear();
     }
 
-    return proto;
-  }
+    builder.setNumRows(this.numRows);
+    builder.setNumBytes(this.numBytes);
 
-  private void mergeLocalToBuilder() {
-    if (builder == null) {
-      builder = TableStatProto.newBuilder(proto);
-    }
-    if (this.numRows != null) {
-      builder.setNumRows(this.numRows);
-    }
-    if (this.numBytes != null) {
-      builder.setNumBytes(this.numBytes);
-    }
     if (this.numBlocks != null) {
       builder.setNumBlocks(this.numBlocks);
     }
@@ -281,5 +192,6 @@ public class TableStat implements ProtoObject<TableStatProto>, Cloneable {
         builder.addColStat(colStat.getProto());
       }
     }
+    return builder.build();
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
index 794176e..dfea828 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
@@ -18,12 +18,12 @@
 
 package org.apache.tajo.catalog;
 
-import com.google.gson.Gson;
-import org.junit.Before;
-import org.junit.Test;
-import org.apache.tajo.catalog.json.GsonCreator;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
+import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.apache.tajo.common.TajoDataTypes.DataType;
 import org.apache.tajo.common.TajoDataTypes.Type;
+import org.junit.Before;
+import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -86,11 +86,17 @@ public class TestColumn {
 	@Test
 	public final void testToJson() {
 		Column col = new Column(field1.getProto());
-		String json = col.toJSON();
+		String json = col.toJson();
 		System.out.println(json);
-		Gson gson = GsonCreator.getInstance();
-		Column fromJson = gson.fromJson(json, Column.class);
-		assertEquals(col.getColumnName(), fromJson.getColumnName());
-		assertEquals(col.getDataType(), fromJson.getDataType());
+		Column fromJson = CatalogGsonHelper.fromJson(json, Column.class);
+		assertEquals(col, fromJson);
 	}
+
+  @Test
+  public final void testToProto() {
+    Column column = new Column(field1.getProto());
+    CatalogProtos.ColumnProto proto = column.getProto();
+    Column fromProto = new Column(proto);
+    assertEquals(column, fromProto);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
index 3546612..936e8c1 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java
@@ -18,10 +18,8 @@
 
 package org.apache.tajo.catalog;
 
-import com.google.gson.Gson;
-import org.junit.Test;
 import org.apache.tajo.catalog.function.GeneralFunction;
-import org.apache.tajo.catalog.json.GsonCreator;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionDescProto;
 import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
 import org.apache.tajo.common.TajoDataTypes;
@@ -32,6 +30,7 @@ import org.apache.tajo.exception.InternalException;
 import org.apache.tajo.storage.Tuple;
 import org.apache.tajo.util.CommonTestingUtil;
 import org.apache.tajo.util.FileUtil;
+import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
@@ -58,13 +57,13 @@ public class TestFunctionDesc {
     }
 
     public String toJSON() {
-      return GsonCreator.getInstance().toJson(this, GeneralFunction.class);
+      return CatalogGsonHelper.toJson(this, GeneralFunction.class);
     }
   }
 
 
   @Test
-  public void testGetSignature() throws IOException {
+  public void testGetSignature() throws IOException, ClassNotFoundException {
     FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
         CatalogUtil.newDataTypesWithoutLen(Type.INT4),
         CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8));
@@ -94,23 +93,25 @@ public class TestFunctionDesc {
   }
   
   @Test
-  public void testJson() throws InternalException {
+  public void testToJson() throws InternalException {
 	  FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
         CatalogUtil.newDataTypesWithoutLen(Type.INT4),
         CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8));
-	  String json = desc.toJSON();
-	  System.out.println(json);
-	  Gson gson = GsonCreator.getInstance();
-	  FunctionDesc fromJson = gson.fromJson(json, FunctionDesc.class);
-	  
-	  assertEquals("sum", fromJson.getSignature());
-	    assertEquals(TestSum.class, fromJson.getFuncClass());
-	    assertEquals(FunctionType.GENERAL, fromJson.getFuncType());
-	    assertEquals(Type.INT4, fromJson.getReturnType()[0].getType());
-	    assertArrayEquals(CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8),
-	    		fromJson.getParamTypes());
-
-	    assertEquals(desc.getProto(), fromJson.getProto());
+	  String json = desc.toJson();
+	  FunctionDesc fromJson = CatalogGsonHelper.fromJson(json, FunctionDesc.class);
+	  assertEquals(desc, fromJson);
+	  assertEquals(desc.getProto(), fromJson.getProto());
+  }
+
+  @Test
+  public void testGetProto() throws InternalException, ClassNotFoundException {
+    FunctionDesc desc = new FunctionDesc("sum", TestSum.class, FunctionType.GENERAL,
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4),
+        CatalogUtil.newDataTypesWithoutLen(Type.INT4, Type.INT8));
+    FunctionDescProto proto = desc.getProto();
+    FunctionDesc fromProto = new FunctionDesc(proto);
+    assertEquals(desc, fromProto);
+    assertEquals(desc.toJson(), fromProto.toJson());
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
index 904d4b7..689f004 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
@@ -18,11 +18,10 @@
 
 package org.apache.tajo.catalog;
 
-import com.google.gson.Gson;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.junit.Before;
 import org.junit.Test;
 import org.apache.tajo.catalog.exception.AlreadyExistsFieldException;
-import org.apache.tajo.catalog.json.GsonCreator;
 import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
 import org.apache.tajo.common.TajoDataTypes.Type;
 
@@ -123,11 +122,16 @@ public class TestSchema {
 	public final void testJson() {
 		Schema schema2 = new Schema(schema.getProto());
 		String json = schema2.toJson();
-		System.out.println(json);
-		Gson gson = GsonCreator.getInstance();
-		Schema fromJson = gson.fromJson(json, Schema.class);
-		assertEquals(schema2.getProto(), fromJson.getProto());
-		assertEquals(schema2.getColumn(0), fromJson.getColumn(0));
-		assertEquals(schema2.getColumnNum(), fromJson.getColumnNum());
+		Schema fromJson = CatalogGsonHelper.fromJson(json, Schema.class);
+		assertEquals(schema2, fromJson);
+    assertEquals(schema2.getProto(), fromJson.getProto());
 	}
+
+  @Test
+  public final void testProto() {
+    Schema schema2 = new Schema(schema.getProto());
+    SchemaProto proto = schema2.getProto();
+    Schema fromJson = new Schema(proto);
+    assertEquals(schema2, fromJson);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
index 4816c45..5235af1 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableDesc.java
@@ -18,11 +18,11 @@
 
 package org.apache.tajo.catalog;
 
-import com.google.gson.Gson;
 import org.apache.hadoop.fs.Path;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
+import org.apache.tajo.catalog.proto.CatalogProtos;
 import org.junit.Before;
 import org.junit.Test;
-import org.apache.tajo.catalog.json.GsonCreator;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.common.TajoDataTypes.Type;
 
@@ -50,50 +50,41 @@ public class TestTableDesc {
     TableMeta info = CatalogUtil.newTableMeta(schema, StoreType.CSV);
     testClone(info);
 
-    TableDesc desc = new TableDescImpl("table1", info, new Path("/nta/data"));
+    TableDesc desc = new TableDescImpl("table1", info, new Path("/tajo/data"));
     assertEquals("table1", desc.getId());
     
-    assertEquals(new Path("/nta/data"), desc.getPath());
+    assertEquals(new Path("/tajo/data"), desc.getPath());
     assertEquals(info, desc.getMeta());
     testClone(desc);
   }
-  
+
   @Test
-  public void testTableMetaToJson() throws CloneNotSupportedException {
-    TableMeta meta = new TableMetaImpl(info.getProto());
-    Gson gson = GsonCreator.getInstance();
-    String json = meta.toJSON();
-    System.out.println(json);
-    TableMeta jsonMeta = gson.fromJson(json, TableMeta.class);
-    assertEquals(meta.getSchema(), jsonMeta.getSchema());
-    assertEquals(meta.getStoreType(), jsonMeta.getStoreType());
-    assertEquals(meta, jsonMeta);
-    testClone(meta);
+  public void testGetProto() throws CloneNotSupportedException {
+    TableDesc desc = new TableDescImpl("table1", info, new Path("/tajo/data"));
+    CatalogProtos.TableDescProto proto = (CatalogProtos.TableDescProto) desc.getProto();
+
+    TableDesc fromProto = new TableDescImpl(proto);
+    assertEquals("equality check the object deserialized from json", desc, fromProto);
   }
-  
-  @Test
-  public void testTableDescToJson() throws CloneNotSupportedException {
-    Gson gson = GsonCreator.getInstance();
 
-    TableDesc desc = new TableDescImpl("table1", info, new Path("/nta/data"));
-    testClone(desc);
+  @Test
+  public void testToJson() throws CloneNotSupportedException {
+    TableDesc desc = new TableDescImpl("table1", info, new Path("/tajo/data"));
+    String json = desc.toJson();
 
-    String json = desc.toJSON();
-    System.out.println(json);
-    TableDesc fromJson = gson.fromJson(json, TableDesc.class);
-    assertEquals(desc.getId(), fromJson.getId());
-    assertEquals(desc.getPath(), fromJson.getPath());
-    assertEquals(desc.getMeta(), fromJson.getMeta());
-    testClone(fromJson);
+    TableDesc fromJson = CatalogGsonHelper.fromJson(json, TableDesc.class);
+    assertEquals("equality check the object deserialized from json", desc, fromJson);
+    assertEquals("equality between protos", desc.getProto(), fromJson.getProto());
   }
 
-  public void testClone(TableDesc desc) throws CloneNotSupportedException {
+  public TableDesc testClone(TableDesc desc) throws CloneNotSupportedException {
     TableDesc copy = (TableDesc) desc.clone();
-    assertEquals(desc, copy);
+    assertEquals("clone check", desc, copy);
+    return copy;
   }
   
   public void testClone(TableMeta meta) throws CloneNotSupportedException {
     TableMeta copy = (TableMeta) meta.clone();
-    assertEquals(meta, copy);
+    assertEquals("clone check", meta, copy);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableMeta.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableMeta.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableMeta.java
index 590e343..88bc74f 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableMeta.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestTableMeta.java
@@ -18,11 +18,14 @@
 
 package org.apache.tajo.catalog;
 
-import org.junit.Before;
-import org.junit.Test;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
 import org.apache.tajo.catalog.proto.CatalogProtos.TableProto;
+import org.apache.tajo.catalog.statistics.ColumnStat;
+import org.apache.tajo.catalog.statistics.TableStat;
 import org.apache.tajo.common.TajoDataTypes.Type;
+import org.junit.Before;
+import org.junit.Test;
 
 import static org.junit.Assert.*;
 
@@ -36,6 +39,23 @@ public class TestTableMeta {
     schema.addColumn("name", Type.BLOB);
     schema.addColumn("addr", Type.TEXT);
     meta = CatalogUtil.newTableMeta(schema, StoreType.CSV);
+
+    TableStat stat = new TableStat();
+    stat.setNumRows(957685);
+    stat.setNumBytes(1023234);
+    stat.setNumBlocks(3123);
+    stat.setNumPartitions(5);
+    stat.setAvgRows(80000);
+
+    int numCols = 2;
+    ColumnStat[] cols = new ColumnStat[numCols];
+    for (int i = 0; i < numCols; i++) {
+      cols[i] = new ColumnStat(schema.getColumn(i));
+      cols[i].setNumDistVals(1024 * i);
+      cols[i].setNumNulls(100 * i);
+      stat.addColumnStat(cols[i]);
+    }
+    meta.setStat(stat);
   }
   
   @Test
@@ -106,22 +126,41 @@ public class TestTableMeta {
     schema2.addColumn("name", Type.BLOB);
     schema2.addColumn("addr", Type.TEXT);
     TableMeta meta2 = CatalogUtil.newTableMeta(schema2, StoreType.CSV);
-    
+
+    TableStat stat = new TableStat();
+    stat.setNumRows(957685);
+    stat.setNumBytes(1023234);
+    stat.setNumBlocks(3123);
+    stat.setNumPartitions(5);
+    stat.setAvgRows(80000);
+
+    int numCols = 2;
+    ColumnStat[] cols = new ColumnStat[numCols];
+    for (int i = 0; i < numCols; i++) {
+      cols[i] = new ColumnStat(schema2.getColumn(i));
+      cols[i].setNumDistVals(1024 * i);
+      cols[i].setNumNulls(100 * i);
+      stat.addColumnStat(cols[i]);
+    }
+    meta2.setStat(stat);
+
+
     assertTrue(meta.equals(meta2));
-    
     assertNotSame(meta, meta2);
   }
   
   @Test
-  public void testGetProto() {    
-    Schema schema1 = new Schema();
-    schema1.addColumn("name", Type.BLOB);
-    schema1.addColumn("addr", Type.TEXT);
-    TableMeta meta1 = CatalogUtil.newTableMeta(schema1, StoreType.CSV);
-    
-    TableProto proto = meta1.getProto();
+  public void testGetProto() {
+    TableProto proto = meta.getProto();
     TableMeta newMeta = new TableMetaImpl(proto);
-    
-    assertTrue(meta1.equals(newMeta));
-  }   
+    assertEquals(meta, newMeta);
+  }
+
+  @Test
+  public void testToJson() {
+    String json = meta.toJson();
+    TableMeta fromJson = CatalogGsonHelper.fromJson(json, TableMeta.class);
+    assertEquals(meta, fromJson);
+    assertEquals(meta.getProto(), fromJson.getProto());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java
index 2f09358..c9dea3c 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java
@@ -18,10 +18,11 @@
 
 package org.apache.tajo.catalog.statistics;
 
-import org.junit.Test;
 import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.datum.DatumFactory;
+import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -55,6 +56,19 @@ public class TestColumnStat {
   }
 
   @Test
+  public final void testJson() throws CloneNotSupportedException {
+    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
+    stat.setNumDistVals(1000);
+    stat.setNumNulls(999);
+    stat.setMinValue(DatumFactory.createInt8(5));
+    stat.setMaxValue(DatumFactory.createInt8(10));
+
+    String json = stat.toJson();
+    ColumnStat fromJson = CatalogGsonHelper.fromJson(json, ColumnStat.class);
+    assertEquals(stat, fromJson);
+  }
+
+  @Test
   public final void testClone() throws CloneNotSupportedException {
     ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
     stat.setNumDistVals(1000);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java
index 6fdd7e2..5372291 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java
@@ -18,9 +18,10 @@
 
 package org.apache.tajo.catalog.statistics;
 
-import org.junit.Test;
 import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.json.CatalogGsonHelper;
 import org.apache.tajo.common.TajoDataTypes.Type;
+import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -58,7 +59,11 @@ public class TestTableStat {
     tableStatEquals(stat, stat2);
     
     TableStat stat3 = (TableStat) stat.clone();
-    tableStatEquals(stat, stat3);    
+    tableStatEquals(stat, stat3);
+
+    String json = stat.toJson();
+    TableStat fromJson = CatalogGsonHelper.fromJson(json, TableStat.class);
+    tableStatEquals(stat, fromJson);
   }
   
   public void tableStatEquals(TableStat s1, TableStat s2) {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java
deleted file mode 100644
index 2f09358..0000000
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestColumnStat.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.catalog.statistics;
-
-import org.junit.Test;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.datum.DatumFactory;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestColumnStat {
-
-  @Test
-  public final void testColumnStat() {
-    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
-    stat.setNumDistVals(1000);
-    stat.setNumNulls(999);
-    
-    assertTrue(1000 == stat.getNumDistValues());
-    assertTrue(999 == stat.getNumNulls());
-    
-    ColumnStat stat2 = new ColumnStat(stat.getProto());
-    assertTrue(1000 == stat2.getNumDistValues());
-    assertTrue(999 == stat2.getNumNulls());
-  }
-
-  @Test
-  public final void testEqualsObject() {
-    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
-    stat.setNumDistVals(1000);
-    stat.setNumNulls(999);
-    stat.setMinValue(DatumFactory.createInt8(5));
-    stat.setMaxValue(DatumFactory.createInt8(10));
-    
-    ColumnStat stat2 = new ColumnStat(stat.getProto());
-    assertEquals(stat, stat2);
-  }
-
-  @Test
-  public final void testClone() throws CloneNotSupportedException {
-    ColumnStat stat = new ColumnStat(new Column("test", Type.INT8));
-    stat.setNumDistVals(1000);
-    stat.setNumNulls(999);
-    stat.setMinValue(DatumFactory.createInt8(5));
-    stat.setMaxValue(DatumFactory.createInt8(10));
-    
-    ColumnStat stat2 = (ColumnStat) stat.clone();
-    assertEquals(stat, stat2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatSet.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatSet.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatSet.java
deleted file mode 100644
index 9285c8d..0000000
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatSet.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.catalog.statistics;
-
-import org.junit.Test;
-import org.apache.tajo.catalog.proto.CatalogProtos.StatType;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestStatSet {
-  @Test
-  public final void testStatGroup() throws CloneNotSupportedException {
-    Stat stat = new Stat(StatType.TABLE_NUM_ROWS);
-    stat.increment();
-    stat.incrementBy(100);
-    assertEquals(101, stat.getValue());
-    
-    Stat stat2 = (Stat) stat.clone();
-    assertEquals(stat, stat2);
-    
-    Stat stat3 = new Stat(StatType.TABLE_NUM_BLOCKS);
-    stat3.increment();
-    stat3.increment();
-    stat3.increment();
-    stat3.subtract();
-    stat3.subtractBy(2);
-    stat3.increment();
-    assertEquals(1, stat3.getValue());
-    
-    StatSet group = new StatSet();
-    group.putStat(stat);
-    group.putStat(stat3);
-    
-    assertEquals(2, group.getAllStats().size());
-    assertEquals(stat, group.getStat(StatType.TABLE_NUM_ROWS));
-    assertEquals(101, group.getStat(StatType.TABLE_NUM_ROWS).getValue());
-    assertEquals(1, group.getStat(StatType.TABLE_NUM_BLOCKS).getValue());
-    
-    StatSet group2 = new StatSet(group.getProto());
-    assertEquals(2, group2.getAllStats().size());
-    assertEquals(stat, group2.getStat(StatType.TABLE_NUM_ROWS));
-    assertEquals(101, group2.getStat(StatType.TABLE_NUM_ROWS).getValue());
-    assertEquals(1, group2.getStat(StatType.TABLE_NUM_BLOCKS).getValue());
-    
-    StatSet group3 = (StatSet) group.clone();
-    assertEquals(2, group3.getAllStats().size());
-    assertEquals(stat, group3.getStat(StatType.TABLE_NUM_ROWS));
-    assertEquals(101, group3.getStat(StatType.TABLE_NUM_ROWS).getValue());
-    assertEquals(1, group3.getStat(StatType.TABLE_NUM_BLOCKS).getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatisticsUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatisticsUtil.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatisticsUtil.java
deleted file mode 100644
index 8019fec..0000000
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestStatisticsUtil.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.catalog.statistics;
-
-import com.google.common.collect.Lists;
-import org.junit.Test;
-import org.apache.tajo.catalog.proto.CatalogProtos.StatType;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-public class TestStatisticsUtil {
-  @Test
-  public void testAggregate() throws CloneNotSupportedException {
-    Stat stat = new Stat(StatType.TABLE_NUM_ROWS);
-    stat.incrementBy(100); // 100
-    assertEquals(100, stat.getValue());
-    
-    Stat stat2 = (Stat) stat.clone();
-    stat2.incrementBy(100); // 200
-    assertEquals(200, stat2.getValue());
-    
-    Stat stat3 = new Stat(StatType.TABLE_NUM_BLOCKS);
-    stat3.incrementBy(50); // 50
-    assertEquals(50, stat3.getValue());
-    
-    StatSet group = new StatSet();
-    group.putStat(stat); // num of rows - 100 
-    group.putStat(stat2); // num of rows - 200
-    group.putStat(stat3); // num of blocks - 50
-    
-    // One group has 300 rows and 50 blocks, and it is cloned.
-    StatSet group2 = (StatSet) group.clone();
-    group2.getStat(StatType.TABLE_NUM_ROWS).incrementBy(100); // plus 100
-    
-    // expected that num of rows = 200 * 2 + 100, num of blocks = 50 * 2 
-    StatSet agg = StatisticsUtil.aggregateStatSet(
-        Lists.newArrayList(group, group2));
-    assertEquals(500, agg.getStat(StatType.TABLE_NUM_ROWS).getValue());
-    assertEquals(100, agg.getStat(StatType.TABLE_NUM_BLOCKS).getValue());
-  }
-
-  @Test
-  public void testEmptyAggregate() {
-    TableStat stat1 = new TableStat();
-    TableStat stat2 = new TableStat();
-    TableStat stat3 = new TableStat();
-
-    assertNotNull(StatisticsUtil.aggregateTableStat(
-        Lists.newArrayList(stat1, stat2, stat3)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java
deleted file mode 100644
index 6fdd7e2..0000000
--- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/statistics/TestTableStat.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.catalog.statistics;
-
-import org.junit.Test;
-import org.apache.tajo.catalog.Column;
-import org.apache.tajo.common.TajoDataTypes.Type;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestTableStat {
-  @Test
-  public final void testTableStat() throws CloneNotSupportedException {
-    TableStat stat = new TableStat();
-    stat.setNumRows(957685);
-    stat.setNumBytes(1023234);
-    stat.setNumBlocks(3123);
-    stat.setNumPartitions(5);
-    stat.setAvgRows(80000);
-        
-    int numCols = 3;
-    ColumnStat[] cols = new ColumnStat[numCols];
-    for (int i = 0; i < numCols; i++) {
-      cols[i] = new ColumnStat(new Column("col_" + i, Type.INT8));
-      cols[i].setNumDistVals(1024 * i);
-      cols[i].setNumNulls(100 * i);
-      stat.addColumnStat(cols[i]);
-    }
-    
-    assertTrue(957685 == stat.getNumRows());
-    assertTrue(1023234 == stat.getNumBytes());
-    assertTrue(3123 == stat.getNumBlocks());
-    assertTrue(5 == stat.getNumPartitions());
-    assertTrue(80000 == stat.getAvgRows());
-    assertEquals(3, stat.getColumnStats().size());
-    for (int i = 0; i < numCols; i++) {
-      assertEquals(cols[i], stat.getColumnStats().get(i));
-    }
-    
-    TableStat stat2 = new TableStat(stat.getProto());
-    tableStatEquals(stat, stat2);
-    
-    TableStat stat3 = (TableStat) stat.clone();
-    tableStatEquals(stat, stat3);    
-  }
-  
-  public void tableStatEquals(TableStat s1, TableStat s2) {
-    assertEquals(s1.getNumRows(), s2.getNumRows());
-    assertEquals(s1.getNumBlocks(), s2.getNumBlocks());
-    assertEquals(s1.getNumPartitions(), s2.getNumPartitions());
-    assertEquals(s1.getAvgRows(), s2.getAvgRows());
-    assertEquals(s1.getColumnStats().size(), s2.getColumnStats().size());
-    for (int i = 0; i < s1.getColumnStats().size(); i++) {
-      assertEquals(s1.getColumnStats().get(i), s2.getColumnStats().get(i));
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java b/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
index 6cea5a9..3ef436e 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryUnitAttemptId.java
@@ -25,8 +25,7 @@ import org.apache.tajo.common.ProtoObject;
 
 import java.text.NumberFormat;
 
-public class QueryUnitAttemptId implements Comparable<QueryUnitAttemptId>,
-    ProtoObject<QueryUnitAttemptIdProto> {
+public class QueryUnitAttemptId implements Comparable<QueryUnitAttemptId>, ProtoObject<QueryUnitAttemptIdProto> {
   private static final String PREFIX="ta";
 
   private static final NumberFormat format = NumberFormat.getInstance();
@@ -133,21 +132,6 @@ public class QueryUnitAttemptId implements Comparable<QueryUnitAttemptId>,
     return this.getId() - o.getId();
   }
 
-  private void mergeProtoToLocal() {
-    QueryUnitAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
-    if (queryUnitId == null) {
-      queryUnitId = new QueryUnitId(p.getQueryUnitId());
-    }
-    if (id == -1) {
-      id = p.getId();
-    }
-  }
-
-  @Override
-  public void initFromProto() {
-    mergeProtoToLocal();
-  }
-
   private void mergeLocalToBuilder() {
     if (builder == null) {
       builder = QueryUnitAttemptIdProto.newBuilder(proto);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java b/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
index 21ca5bb..4826691 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryUnitId.java
@@ -44,7 +44,7 @@ public class QueryUnitId implements Comparable<QueryUnitId>,
   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;
@@ -138,21 +138,6 @@ public class QueryUnitId implements Comparable<QueryUnitId>,
     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);

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/common/ProtoObject.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/common/ProtoObject.java b/tajo-common/src/main/java/org/apache/tajo/common/ProtoObject.java
index 2c56e01..d5f7533 100644
--- a/tajo-common/src/main/java/org/apache/tajo/common/ProtoObject.java
+++ b/tajo-common/src/main/java/org/apache/tajo/common/ProtoObject.java
@@ -21,6 +21,5 @@ package org.apache.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/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/ArrayDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/ArrayDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/ArrayDatum.java
index a8578c9..24c38c4 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/ArrayDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/ArrayDatum.java
@@ -19,7 +19,6 @@
 package org.apache.tajo.datum;
 
 import com.google.gson.annotations.Expose;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import static org.apache.tajo.common.TajoDataTypes.Type;
 
@@ -73,9 +72,4 @@ public class ArrayDatum extends Datum {
 
     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/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/BitDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/BitDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/BitDatum.java
index da40a2e..f5c60f6 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/BitDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/BitDatum.java
@@ -21,7 +21,6 @@ package org.apache.tajo.datum;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 public class BitDatum extends Datum {
   private static final int size = 1;
@@ -81,10 +80,6 @@ public class BitDatum extends Datum {
 	public String asChars() {
 		return "0x"+val;
 	}
-	
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
 
   @Override
   public int size() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java
index 4bc4c61..c9ecced 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java
@@ -23,7 +23,6 @@ package org.apache.tajo.datum;
 
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
@@ -109,11 +108,6 @@ public class BlobDatum extends Datum {
 	}
 
   @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
   public int size() {
 	  return this.val.length;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/BooleanDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/BooleanDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/BooleanDatum.java
index adb332a..fc543b4 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/BooleanDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/BooleanDatum.java
@@ -21,7 +21,6 @@ package org.apache.tajo.datum;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 public class BooleanDatum extends Datum {
 	@Expose private boolean val;
@@ -120,10 +119,6 @@ public class BooleanDatum extends Datum {
 	public String asChars() {
 		return val ? "true" : "false";
 	}
-	
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
 
   @Override
   public int size() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/CharDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/CharDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/CharDatum.java
index 8277849..4d77dad 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/CharDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/CharDatum.java
@@ -20,7 +20,6 @@ package org.apache.tajo.datum;
 
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import static org.apache.tajo.common.TajoDataTypes.Type;
 
@@ -89,11 +88,6 @@ public class CharDatum extends Datum {
 	}
 
   @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
   public int size() {
     return size;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
index 92100f6..10ae9d9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Datum.java
@@ -19,14 +19,15 @@
 package org.apache.tajo.datum;
 
 import com.google.gson.annotations.Expose;
+import org.apache.tajo.json.GsonObject;
 import org.apache.tajo.datum.exception.InvalidCastException;
 import org.apache.tajo.datum.exception.InvalidOperationException;
+import org.apache.tajo.json.CommonGsonHelper;
 
 import static org.apache.tajo.common.TajoDataTypes.Type;
 
-public abstract class Datum implements Comparable<Datum> {
-	@Expose
-	private Type type;
+public abstract class Datum implements Comparable<Datum>, GsonObject {
+	@Expose	private Type type;
 	
 	@SuppressWarnings("unused")
   private Datum() {
@@ -149,7 +150,10 @@ public abstract class Datum implements Comparable<Datum> {
 	
   public abstract int compareTo(Datum datum);
 
-  public abstract String toJSON();
+  @Override
+  public String toJson() {
+    return CommonGsonHelper.toJson(this, Datum.class);
+  }
 
   @Override
   public String toString() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/Float4Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Float4Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Float4Datum.java
index cbe9206..3bdd332 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Float4Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Float4Datum.java
@@ -22,7 +22,6 @@ import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.exception.InvalidCastException;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import java.nio.ByteBuffer;
 
@@ -92,11 +91,6 @@ public class Float4Datum extends NumericDatum {
 	}
 
   @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
   public int size() {
     return size;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/Float8Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Float8Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Float8Datum.java
index a75f042..a27b4fc 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Float8Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Float8Datum.java
@@ -24,7 +24,6 @@ package org.apache.tajo.datum;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import java.nio.ByteBuffer;
 
@@ -85,11 +84,6 @@ public class Float8Datum extends NumericDatum {
 	}
 
   @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
   public int size() {
     return size;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/Inet4Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Inet4Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Inet4Datum.java
index f039220..62f510d 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Inet4Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Inet4Datum.java
@@ -21,7 +21,6 @@ package org.apache.tajo.datum;
 import com.google.common.base.Preconditions;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import static org.apache.tajo.common.TajoDataTypes.Type;
 
@@ -77,11 +76,6 @@ public class Inet4Datum extends Datum {
 	}
 
   @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
   public int size() {
     return size;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/Int2Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Int2Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Int2Datum.java
index 92a867f..7523432 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Int2Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Int2Datum.java
@@ -21,7 +21,6 @@ package org.apache.tajo.datum;
 import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import java.nio.ByteBuffer;
 
@@ -82,11 +81,6 @@ public class Int2Datum extends NumericDatum {
 	}
 
   @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
   public int size() {
     return size;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/Int4Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Int4Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Int4Datum.java
index 38f3588..0c993a3 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Int4Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Int4Datum.java
@@ -22,7 +22,6 @@ import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.TajoDataTypes.Type;
 import org.apache.tajo.datum.exception.InvalidCastException;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import java.nio.ByteBuffer;
 
@@ -88,11 +87,6 @@ public class Int4Datum extends NumericDatum {
 	}
 
   @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
-
-  @Override
   public int size() {
     return size;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/Int8Datum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/Int8Datum.java b/tajo-common/src/main/java/org/apache/tajo/datum/Int8Datum.java
index fc02b16..f93c219 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/Int8Datum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/Int8Datum.java
@@ -22,7 +22,6 @@ import com.google.gson.annotations.Expose;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.exception.InvalidCastException;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import java.nio.ByteBuffer;
 
@@ -91,11 +90,6 @@ public class Int8Datum extends NumericDatum {
 	public String asChars() {
 		return ""+val;
 	}
-
-  @Override
-	public String toJSON() {
-		return GsonCreator.getInstance().toJson(this, Datum.class);
-	}
 	
   @Override
   public int size() {

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
index 222a81e..334e935 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/NullDatum.java
@@ -101,9 +101,4 @@ public class NullDatum extends Datum {
   public int hashCode() {
     return 23244; // one of the prime number
   }
-
-  @Override
-  public String toJSON() {
-    return "";
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
index 8612a35..876803b 100644
--- a/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
+++ b/tajo-common/src/main/java/org/apache/tajo/datum/TextDatum.java
@@ -23,15 +23,12 @@ import org.apache.hadoop.io.WritableComparator;
 import org.apache.tajo.common.TajoDataTypes;
 import org.apache.tajo.datum.exception.InvalidCastException;
 import org.apache.tajo.datum.exception.InvalidOperationException;
-import org.apache.tajo.datum.json.GsonCreator;
 
 import java.util.Arrays;
 
 public class TextDatum extends Datum {
-  @Expose
-  private int size;
-  @Expose
-  private byte[] bytes;
+  @Expose private int size;
+  @Expose private byte [] bytes;
 
   public TextDatum() {
     super(TajoDataTypes.Type.TEXT);
@@ -130,11 +127,6 @@ public class TextDatum extends Datum {
   }
 
   @Override
-  public String toJSON() {
-    return GsonCreator.getInstance().toJson(this, Datum.class);
-  }
-
-  @Override
   public int hashCode() {
     return Arrays.hashCode(bytes);
   }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/json/DatumAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/json/DatumAdapter.java b/tajo-common/src/main/java/org/apache/tajo/datum/json/DatumAdapter.java
deleted file mode 100644
index dfe0e21..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/datum/json/DatumAdapter.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.datum.json;
-
-import com.google.gson.*;
-import org.apache.tajo.datum.Datum;
-
-import java.lang.reflect.Type;
-
-public class DatumAdapter implements JsonSerializer<Datum>, JsonDeserializer<Datum> {
-
-	@Override
-	public Datum deserialize(JsonElement json, Type typeOfT,
-			JsonDeserializationContext context) throws JsonParseException {
-		JsonObject jsonObject = json.getAsJsonObject();
-		String className = jsonObject.get("classname").getAsJsonPrimitive().getAsString();
-		
-		Class clazz;
-		try {
-			clazz = Class.forName(className);
-		} catch (ClassNotFoundException e) {
-			e.printStackTrace();
-			throw new JsonParseException(e);
-		}
-		return context.deserialize(jsonObject.get("property"), clazz);
-	}
-
-	@Override
-	public JsonElement serialize(Datum src, Type typeOfSrc,
-			JsonSerializationContext context) {
-		JsonObject jsonObj = new JsonObject();
-		String className = src.getClass().getCanonicalName();
-		jsonObj.addProperty("classname", className);
-		JsonElement jsonElem = context.serialize(src);
-		jsonObj.add("property", jsonElem);
-		return jsonObj;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/datum/json/GsonCreator.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/json/GsonCreator.java b/tajo-common/src/main/java/org/apache/tajo/datum/json/GsonCreator.java
deleted file mode 100644
index d1a9bf2..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/datum/json/GsonCreator.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.datum.json;
-
-import com.google.gson.Gson;
-import com.google.gson.GsonBuilder;
-import org.apache.tajo.datum.Datum;
-
-public class GsonCreator {
-
-	private static GsonBuilder builder;
-	private static Gson gson;
-	
-	private static void init() {
-		if (builder == null) {
-			builder = new GsonBuilder().excludeFieldsWithoutExposeAnnotation();
-			builder.registerTypeAdapter(Datum.class, new DatumAdapter());
-		} 
-		if (gson == null ) {
-			gson = builder.create();
-		}
-	}
-
-	public static Gson getInstance() {
-		init();
-		return gson;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameDeserializer.java b/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameDeserializer.java
deleted file mode 100644
index 43a378e..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameDeserializer.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package org.apache.tajo.gson;
-
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonParseException;
-
-import java.lang.reflect.Type;
-
-public class ClassNameDeserializer implements JsonDeserializer<Class> {
-
-	@Override
-	public Class deserialize(JsonElement json, Type type,
-			JsonDeserializationContext ctx) throws JsonParseException {
-		try {
-			return Class.forName(json.getAsString());
-		} catch (ClassNotFoundException e) {
-			e.printStackTrace();
-		}
-		return null;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameSerializer.java b/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameSerializer.java
deleted file mode 100644
index 7cc9245..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/gson/ClassNameSerializer.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package org.apache.tajo.gson;
-
-import com.google.gson.JsonElement;
-import com.google.gson.JsonPrimitive;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
-
-import java.lang.reflect.Type;
-
-public class ClassNameSerializer implements JsonSerializer<Class> {
-
-	@Override
-	public JsonElement serialize(Class clazz, Type type,
-			JsonSerializationContext ctx) {
-		return new JsonPrimitive(clazz.getName());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/gson/DataTypeAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/gson/DataTypeAdapter.java b/tajo-common/src/main/java/org/apache/tajo/gson/DataTypeAdapter.java
deleted file mode 100644
index 9fc0586..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/gson/DataTypeAdapter.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.gson;
-
-import com.google.gson.*;
-import org.apache.tajo.common.TajoDataTypes;
-import org.apache.tajo.common.TajoDataTypes.DataType;
-
-import java.lang.reflect.Type;
-
-
-public class DataTypeAdapter implements JsonSerializer<DataType>, JsonDeserializer<DataType> {
-
-  @Override
-  public DataType deserialize(JsonElement json, Type typeOfT, JsonDeserializationContext context)
-      throws JsonParseException {
-
-
-    JsonObject obj = (JsonObject) json;
-    DataType.Builder builder = DataType.newBuilder();
-    TajoDataTypes.Type type = Enum.valueOf(TajoDataTypes.Type.class, obj.get("type").getAsString());
-    builder.setType(type);
-
-    JsonElement len = obj.get("len");
-    if (len != null) {
-      builder.setLength(len.getAsInt());
-    }
-    JsonElement code = obj.get("code");
-    if (code != null) {
-      builder.setCode(code.getAsString());
-    }
-    return builder.build();
-  }
-
-  @Override
-  public JsonElement serialize(DataType src, Type typeOfSrc, JsonSerializationContext context) {
-    JsonObject json = new JsonObject();
-    json.addProperty("type", src.getType().name());
-    if (src.hasLength()) {
-      json.addProperty("len", src.getLength());
-    }
-    if (src.hasCode()) {
-      json.addProperty("code", src.getCode());
-    }
-
-    return json;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/gson/DatumTypeAdapter.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/gson/DatumTypeAdapter.java b/tajo-common/src/main/java/org/apache/tajo/gson/DatumTypeAdapter.java
deleted file mode 100644
index b457fc1..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/gson/DatumTypeAdapter.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.gson;
-
-import com.google.gson.*;
-import org.apache.tajo.datum.Datum;
-
-import java.lang.reflect.Type;
-
-public class DatumTypeAdapter implements JsonSerializer<Datum>, JsonDeserializer<Datum> {
-
-  @Override
-  public Datum deserialize(JsonElement json, Type typeOfT,
-      JsonDeserializationContext context) throws JsonParseException {
-    JsonObject jsonObject = json.getAsJsonObject();
-    String className = jsonObject.get("classname").getAsJsonPrimitive().getAsString();
-    
-    Class clazz;
-    try {
-      clazz = Class.forName(className);
-    } catch (ClassNotFoundException e) {
-      e.printStackTrace();
-      throw new JsonParseException(e);
-    }
-    return context.deserialize(jsonObject.get("property"), clazz);
-  }
-
-  @Override
-  public JsonElement serialize(Datum src, Type typeOfSrc,
-      JsonSerializationContext context) {
-    JsonObject jsonObj = new JsonObject();
-    String className = src.getClass().getCanonicalName();
-    jsonObj.addProperty("classname", className);
-    JsonElement jsonElem = context.serialize(src);
-    jsonObj.add("property", jsonElem);
-    return jsonObj;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/gson/PathDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/gson/PathDeserializer.java b/tajo-common/src/main/java/org/apache/tajo/gson/PathDeserializer.java
deleted file mode 100644
index ae3eb81..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/gson/PathDeserializer.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package org.apache.tajo.gson;
-
-import com.google.gson.JsonDeserializationContext;
-import com.google.gson.JsonDeserializer;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonParseException;
-import org.apache.hadoop.fs.Path;
-
-import java.lang.reflect.Type;
-
-public class PathDeserializer implements JsonDeserializer<Path> {
-
-	@Override
-	public Path deserialize(JsonElement arg0, Type arg1,
-			JsonDeserializationContext arg2) throws JsonParseException {
-		return new Path(arg0.getAsJsonPrimitive().getAsString());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/gson/PathSerializer.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/gson/PathSerializer.java b/tajo-common/src/main/java/org/apache/tajo/gson/PathSerializer.java
deleted file mode 100644
index 6564d7d..0000000
--- a/tajo-common/src/main/java/org/apache/tajo/gson/PathSerializer.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * 
- */
-package org.apache.tajo.gson;
-
-import com.google.gson.JsonElement;
-import com.google.gson.JsonPrimitive;
-import com.google.gson.JsonSerializationContext;
-import com.google.gson.JsonSerializer;
-import org.apache.hadoop.fs.Path;
-
-import java.lang.reflect.Type;
-
-public class PathSerializer implements JsonSerializer<Path> {
-
-	@Override
-	public JsonElement serialize(Path arg0, Type arg1,
-			JsonSerializationContext arg2) {
-		return new JsonPrimitive(arg0.toString());
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1c677cc5/tajo-common/src/main/java/org/apache/tajo/json/ClassNameDeserializer.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/json/ClassNameDeserializer.java b/tajo-common/src/main/java/org/apache/tajo/json/ClassNameDeserializer.java
new file mode 100644
index 0000000..d069db2
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/json/ClassNameDeserializer.java
@@ -0,0 +1,44 @@
+/**
+ * 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.tajo.json;
+
+import com.google.gson.JsonDeserializationContext;
+import com.google.gson.JsonDeserializer;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonParseException;
+
+import java.lang.reflect.Type;
+
+public class ClassNameDeserializer implements JsonDeserializer<Class> {
+
+	@Override
+	public Class deserialize(JsonElement json, Type type,
+			JsonDeserializationContext ctx) throws JsonParseException {
+		try {
+			return Class.forName(json.getAsString());
+		} catch (ClassNotFoundException e) {
+			e.printStackTrace();
+		}
+		return null;
+	}
+
+}