You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by rv...@apache.org on 2015/09/19 02:36:18 UTC

[35/51] [partial] incubator-hawq git commit: SGA import

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_Parquet_Types.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_Parquet_Types.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_Parquet_Types.java
new file mode 100644
index 0000000..afbc167
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_Parquet_Types.java
@@ -0,0 +1,182 @@
+package com.pivotal.hawq.mapreduce.ft;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.SimpleTableClusterTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test data type support for Parquet table.
+ */
+public class HAWQInputFormatFeatureTest_Parquet_Types extends SimpleTableClusterTester {
+
+	@BeforeClass
+	public static void setUpBeforeClass() {
+		System.out.println("Executing test suite: Parquet_Types");
+	}
+
+	private void doSingleColumnParquetTest(String columnType) throws Exception {
+		String tableName = "test_parquet_types_" + columnType;
+		tableName = tableName.replace("(", "")
+							 .replace(")", "")
+							 .replace("[]", "_array");
+
+		HAWQTable table = new HAWQTable.Builder(tableName, Lists.newArrayList(columnType))
+				.storage(HAWQTableFormat.Parquet)
+				.build();
+
+		testSimpleTable(table);
+	}
+
+	@Test
+	public void testBool() throws Exception {
+		doSingleColumnParquetTest("bool");
+	}
+
+	@Test
+	public void testBit() throws Exception {
+		doSingleColumnParquetTest("bit");
+	}
+
+	@Test
+	public void testBitN() throws Exception {
+		doSingleColumnParquetTest("bit(5)");
+	}
+
+	@Test
+	public void testVarbit() throws Exception {
+		doSingleColumnParquetTest("varbit");
+	}
+
+	@Test
+	public void testByteArray() throws Exception {
+		doSingleColumnParquetTest("bytea");
+	}
+
+	@Test
+	public void testInt2() throws Exception {
+		doSingleColumnParquetTest("int2");
+	}
+
+	@Test
+	public void testInt4() throws Exception {
+		doSingleColumnParquetTest("int4");
+	}
+
+	@Test
+	public void testInt8() throws Exception {
+		doSingleColumnParquetTest("int8");
+	}
+
+	@Test
+	public void testFloat4() throws Exception {
+		doSingleColumnParquetTest("float4");
+	}
+
+	@Test
+	public void testFloat8() throws Exception {
+		doSingleColumnParquetTest("float8");
+	}
+
+	@Test
+	public void testNumeric() throws Exception {
+		doSingleColumnParquetTest("numeric");
+	}
+
+	@Test
+	public void testCharN() throws Exception {
+		doSingleColumnParquetTest("char(10)");
+	}
+
+	@Test
+	public void testVarcharN() throws Exception {
+		doSingleColumnParquetTest("varchar(10)");
+	}
+
+	@Test
+	public void testText() throws Exception {
+		doSingleColumnParquetTest("text");
+	}
+
+	@Test
+	public void testDate() throws Exception {
+		doSingleColumnParquetTest("date");
+	}
+
+	@Test
+	public void testTime() throws Exception {
+		doSingleColumnParquetTest("time");
+	}
+
+	@Test
+	public void testTimetz() throws Exception {
+		doSingleColumnParquetTest("timetz");
+	}
+
+	@Test
+	public void testTimestamp() throws Exception {
+		doSingleColumnParquetTest("timestamp");
+	}
+
+	@Test
+	public void testTimestamptz() throws Exception {
+		doSingleColumnParquetTest("timestamptz");
+	}
+
+	@Test
+	public void testInterval() throws Exception {
+		doSingleColumnParquetTest("interval");
+	}
+
+	@Test
+	public void testPoint() throws Exception {
+		doSingleColumnParquetTest("point");
+	}
+
+	@Test
+	public void testLseg() throws Exception {
+		doSingleColumnParquetTest("lseg");
+	}
+
+	@Test
+	public void testBox() throws Exception {
+		doSingleColumnParquetTest("box");
+	}
+
+	@Test
+	public void testCircle() throws Exception {
+		doSingleColumnParquetTest("circle");
+	}
+
+	@Test
+	public void testPath() throws Exception {
+		doSingleColumnParquetTest("path");
+	}
+
+	@Test
+	public void testPolygon() throws Exception {
+		doSingleColumnParquetTest("polygon");
+	}
+
+	@Test
+	public void testMacaddr() throws Exception {
+		doSingleColumnParquetTest("macaddr");
+	}
+
+	@Test
+	public void testInet() throws Exception {
+		doSingleColumnParquetTest("inet");
+	}
+
+	@Test
+	public void testCidr() throws Exception {
+		doSingleColumnParquetTest("cidr");
+	}
+
+	@Test
+	public void testXml() throws Exception {
+		doSingleColumnParquetTest("xml");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_TPCH.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_TPCH.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_TPCH.java
new file mode 100644
index 0000000..dfc5bc6
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ft/HAWQInputFormatFeatureTest_TPCH.java
@@ -0,0 +1,41 @@
+package com.pivotal.hawq.mapreduce.ft;
+
+import com.pivotal.hawq.mapreduce.TPCHClusterTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Test reading TPC-H table using HAWQInputFormat
+ */
+public class HAWQInputFormatFeatureTest_TPCH extends TPCHClusterTester {
+
+	@BeforeClass
+	public static void setUpBeforeClass() {
+		System.out.println("Executing test suite: TPC-H");
+	}
+
+	@Test
+	public void testTPCH_AO_No_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.02", HAWQTableFormat.AO, false);
+		testTPCHTable(tpchSpec, "lineitem_ao_row");
+	}
+
+	@Test
+	public void testTPCH_AO_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.01", HAWQTableFormat.AO, true);
+		testTPCHTable(tpchSpec, "lineitem_ao_row");
+	}
+
+	@Test
+	public void testTPCH_Parquet_No_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.02", HAWQTableFormat.Parquet, false);
+		testTPCHTable(tpchSpec, "lineitem_ao_parquet");
+	}
+
+	@Test
+	public void testTPCH_Parquet_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.01", HAWQTableFormat.Parquet, true);
+		testTPCHTable(tpchSpec, "lineitem_ao_parquet");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/pt/HAWQInputFormatPerformanceTest_TPCH.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/pt/HAWQInputFormatPerformanceTest_TPCH.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/pt/HAWQInputFormatPerformanceTest_TPCH.java
new file mode 100644
index 0000000..3d99e2d
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/pt/HAWQInputFormatPerformanceTest_TPCH.java
@@ -0,0 +1,212 @@
+package com.pivotal.hawq.mapreduce.pt;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.*;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import com.pivotal.hawq.mapreduce.schema.HAWQSchema;
+import com.pivotal.hawq.mapreduce.util.HAWQJdbcUtils;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * MapReduce driver class to do manually performance tests on reading TPC-H table.
+ *
+ * Usage:
+ * 1. load and query
+ * HAWQInputFormatPerformanceTest_TPCH ao|parquet scale is_partition tableName [columns]
+ * 2. query-only
+ * HAWQInputFormatPerformanceTest_TPCH --query-only tableName [columns]
+ *
+ * Example:
+ * $ hadoop com.pivotal.hawq.mapreduce.pt.HAWQInputFormatPerformanceTest_TPCH ao 5 false lineitem_ao_row L_PARTKEY,L_COMMENT
+ */
+public class HAWQInputFormatPerformanceTest_TPCH extends Configured implements Tool {
+
+	static class TPCHTableMapper extends Mapper<Void, HAWQRecord, Text, Void> {
+		@Override
+		protected void map(Void key, HAWQRecord value, Context context)
+				throws IOException, InterruptedException {
+			try {
+				String recordString = toRecordString(value);
+				context.write(new Text(recordString), null);
+
+			} catch (HAWQException e) {
+				throw new IOException(e);
+			}
+		}
+
+		private String toRecordString(HAWQRecord record) throws HAWQException {
+			HAWQSchema schema = record.getSchema();
+
+			// read all columns if user didn't specify column list
+			if (colNames == null) {
+				StringBuilder buf = new StringBuilder(toFieldString(record, 1));
+				for (int i = 2; i <= schema.getFieldCount(); i++) {
+					buf.append("|").append(toFieldString(record, i));
+				}
+				return buf.toString();
+			}
+
+			assert colNames.size() > 0;
+			StringBuilder buf = new StringBuilder(toFieldString(record, schema.getFieldIndex(colNames.get(0))));
+			for (int i = 1; i < colNames.size(); i++) {
+				buf.append("|").append(toFieldString(record, schema.getFieldIndex(colNames.get(i))));
+			}
+			return buf.toString();
+		}
+
+		private String toFieldString(HAWQRecord record, int fieldIndex)
+				throws HAWQException {
+			Object val = record.getObject(fieldIndex);
+			if (val == null) return "null";
+			if (val instanceof byte[]) return new String((byte[]) val);
+			return val.toString();
+		}
+	}
+
+	// command line arguments
+	boolean queryOnly;
+	HAWQTableFormat tableFormat;
+	String scale;
+	boolean isPartition;
+	String tableName;
+	static List<String> colNames;	// used by TPCHTableMapper to determine which column to read.
+
+	// counters
+	long dataLoadTime;
+	long metadataExtractTime;
+	long mapReduceTime;
+
+	private boolean readArguments(String[] args) {
+		if (args.length == 0)
+			return false;
+
+		int tableNameArgIndex;
+		queryOnly = args[0].equalsIgnoreCase("--query-only");
+
+		if (queryOnly) {
+			if (args.length != 2 && args.length != 3)
+				return false;
+			tableNameArgIndex = 1;
+
+		} else {
+			if (args[0].equalsIgnoreCase("ao"))
+				tableFormat = HAWQTableFormat.AO;
+			else if (args[0].equalsIgnoreCase("parquet"))
+				tableFormat = HAWQTableFormat.Parquet;
+			else
+				return false;
+
+			if (args.length != 4 && args.length != 5)
+				return false;
+
+			scale = args[1];
+			isPartition = Boolean.parseBoolean(args[2]);
+			tableNameArgIndex = 3;
+		}
+
+		tableName = args[tableNameArgIndex];
+		if (tableNameArgIndex + 1 < args.length) {
+			colNames = Lists.newArrayList(args[tableNameArgIndex + 1].split(","));
+		}
+		return true;
+	}
+
+	private void loadTPCHData() throws Exception {
+		// get number of segments
+		int segnum;
+		Connection conn = null;
+		try {
+			conn = MRFormatTestUtils.getTestDBConnection();
+			Map<String, String> rs = HAWQJdbcUtils.executeSafeQueryForSingleRow(
+					conn, "SELECT COUNT(*) segnum FROM gp_segment_configuration WHERE content>=0;");
+			segnum = Integer.parseInt(rs.get("segnum"));
+		} finally {
+			HAWQJdbcUtils.closeConnection(conn);
+		}
+
+		// run external script to load TPC-H data
+		TPCHTester.HAWQTPCHSpec spec = new TPCHTester.HAWQTPCHSpec(scale, tableFormat, isPartition);
+		MRFormatTestUtils.runShellCommand(spec.getLoadCmd(segnum));
+	}
+	
+	private int runMapReduceJob() throws Exception {
+		Path outputPath = new Path("/output");
+		// delete previous output
+		FileSystem fs = FileSystem.get(getConf());
+		if (fs.exists(outputPath))
+			fs.delete(outputPath, true);
+		fs.close();
+
+		Job job = new Job(getConf());
+		job.setJarByClass(HAWQInputFormatPerformanceTest_TPCH.class);
+
+		job.setInputFormatClass(HAWQInputFormat.class);
+
+		long startTime = System.currentTimeMillis();
+		HAWQInputFormat.setInput(job.getConfiguration(), MRFormatConfiguration.TEST_DB_URL, null, null, tableName);
+		metadataExtractTime = System.currentTimeMillis() - startTime;
+
+		FileOutputFormat.setOutputPath(job, outputPath);
+
+		job.setMapperClass(TPCHTableMapper.class);
+		job.setNumReduceTasks(0);
+
+		job.setOutputKeyClass(Text.class);
+		job.setOutputValueClass(Void.class);
+
+		return job.waitForCompletion(true) ? 0 : 1;
+	}
+
+	@Override
+	public int run(String[] args) throws Exception {
+		if (!readArguments(args)) {
+			System.err.printf("Usage: %s [generic options] <ao|parquet> <scale> <is_partition> <tableName> [<col1,col2>]\n",
+							  getClass().getSimpleName());
+			System.err.printf("       %s [generic options] --query-only <tableName> [<col1,col2>]\n",
+							  getClass().getSimpleName());
+			ToolRunner.printGenericCommandUsage(System.err);
+			return -1;
+		}
+
+		long startTime;
+
+		if (!queryOnly) {
+			startTime = System.currentTimeMillis();
+			loadTPCHData();
+			dataLoadTime = System.currentTimeMillis() - startTime;
+		}
+
+		startTime = System.currentTimeMillis();
+		int res = runMapReduceJob();
+		mapReduceTime = System.currentTimeMillis() - startTime;
+
+		System.out.println("=====================================");
+		System.out.println("========= Reports ===================");
+		System.out.println("Table read : " + tableName);
+		if (!queryOnly) {
+		System.out.println("Data volume:        " + scale + "G");
+		System.out.println("TPC-H Data Loading: " + dataLoadTime + "ms");
+		}
+		System.out.println("Metadata Extract:   " + metadataExtractTime + "ms");
+		System.out.println("MapReduce Job:      " + mapReduceTime + "ms");
+		return res;
+	}
+
+	public static void main(String[] args) throws Exception {
+		int exitCode = ToolRunner.run(new HAWQInputFormatPerformanceTest_TPCH(), args);
+		System.exit(exitCode);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Compression.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Compression.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Compression.java
new file mode 100644
index 0000000..de8344e
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Compression.java
@@ -0,0 +1,33 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.DataProvider;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.RandomDataProvider;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Test reading AO compressed table.
+ */
+public class HAWQInputFormatUnitTest_AO_Compression extends SimpleTableLocalTester {
+
+	private List<String> colTypes = Lists.newArrayList("int4", "text");
+	private DataProvider provider = new RandomDataProvider(500);
+
+	@Test
+	public void testZlib() throws Exception {
+		for (int compressLevel = 1; compressLevel < 10; compressLevel++) {
+			HAWQTable table = new HAWQTable.Builder("test_ao_zlib_" + compressLevel, colTypes)
+					.storage(HAWQTableFormat.AO)
+					.compress("zlib", compressLevel)
+					.provider(provider)
+					.build();
+
+			testSimpleTable(table);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Misc.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Misc.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Misc.java
new file mode 100644
index 0000000..5f87144
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Misc.java
@@ -0,0 +1,49 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.DataProvider;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Test;
+
+/**
+ * Miscellaneous tests for AO tables.
+ */
+public class HAWQInputFormatUnitTest_AO_Misc extends SimpleTableLocalTester {
+
+	@Test
+	public void testAOLargeContent() throws Exception {
+		HAWQTable table = new HAWQTable.Builder("test_ao_largecontent", Lists.newArrayList("text"))
+				.storage(HAWQTableFormat.AO)
+				.blockSize(32768)    // 32K
+				.provider(new DataProvider() {
+					@Override
+					public String getInsertSQLs(HAWQTable table) {
+						return "INSERT INTO " + table.getTableName() + " values (repeat('b', 40000));";
+					}
+				}).build();
+
+		testSimpleTable(table);
+	}
+
+	@Test
+	public void testAOEmptyTable() throws Exception {
+		HAWQTable table = new HAWQTable.Builder("test_ao_empty", Lists.newArrayList("int4"))
+				.storage(HAWQTableFormat.AO)
+				.provider(DataProvider.EMPTY)
+				.build();
+
+		testSimpleTable(table);
+	}
+
+	@Test
+	public void testAORecordGetAllTypes() throws Exception {
+		HAWQTable table = new HAWQTable.Builder("test_ao_alltypes", UnitTestAllTypesMapper.types)
+				.storage(HAWQTableFormat.AO)
+				.provider(DataProvider.RANDOM)
+				.build();
+
+		testSimpleTable(table, UnitTestAllTypesMapper.class);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Options.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Options.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Options.java
new file mode 100644
index 0000000..6f16b82
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Options.java
@@ -0,0 +1,50 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.SeriesIntDataProvider;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Test reading AO tables with different options.
+ */
+public class HAWQInputFormatUnitTest_AO_Options extends SimpleTableLocalTester {
+
+	@Test
+	public void testBlockSize() throws Exception {
+		List<String> colTypes = Lists.newArrayList("int8");
+
+		int[] blockSizes = {8192, 16384, 65536};
+		for (int blockSize : blockSizes) {
+			String tableName = "test_ao_blocksize_" + blockSize;
+			HAWQTable table = new HAWQTable.Builder(tableName, colTypes)
+					.storage(HAWQTableFormat.AO)
+					.blockSize(blockSize)
+					.provider(new SeriesIntDataProvider(blockSize))
+					.build();
+
+			testSimpleTable(table);
+		}
+	}
+
+	@Test
+	public void testChecksum() throws Exception {
+		List<String> colTypes = Lists.newArrayList("int4", "varchar(10)");
+
+		HAWQTable table = new HAWQTable.Builder("test_ao_with_checksum", colTypes)
+				.storage(HAWQTableFormat.AO)
+				.checkSum(true)
+				.build();
+		testSimpleTable(table);
+
+		table = new HAWQTable.Builder("test_ao_without_checksum", colTypes)
+				.storage(HAWQTableFormat.AO)
+				.checkSum(false)
+				.build();
+		testSimpleTable(table);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Types.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Types.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Types.java
new file mode 100644
index 0000000..dcd29b8
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_AO_Types.java
@@ -0,0 +1,186 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * Test data type support for AO table.
+ */
+public class HAWQInputFormatUnitTest_AO_Types extends SimpleTableLocalTester {
+
+	private void doSingleColumnAOTest(String columnType) throws Exception {
+		String tableName = "test_ao_types_" + columnType;
+		tableName = tableName.replace("(", "")
+							 .replace(")", "")
+							 .replace("[]", "_array");
+
+		HAWQTable table = new HAWQTable.Builder(tableName, Lists.newArrayList(columnType))
+				.storage(HAWQTableFormat.AO)
+				.build();
+
+		testSimpleTable(table);
+	}
+
+	@Test
+	public void testBool() throws Exception {
+		doSingleColumnAOTest("bool");
+	}
+
+	@Test
+	public void testBit() throws Exception {
+		doSingleColumnAOTest("bit");
+	}
+
+	@Test
+	public void testBitN() throws Exception {
+		doSingleColumnAOTest("bit(5)");
+	}
+
+	@Test
+	public void testVarbit() throws Exception {
+		doSingleColumnAOTest("varbit");
+	}
+
+	@Test
+	public void testByteArray() throws Exception {
+		doSingleColumnAOTest("bytea");
+	}
+
+	@Test
+	public void testInt2() throws Exception {
+		doSingleColumnAOTest("int2");
+	}
+
+	@Test
+	public void testInt4() throws Exception {
+		doSingleColumnAOTest("int4");
+	}
+
+	@Test
+	public void testInt8() throws Exception {
+		doSingleColumnAOTest("int8");
+	}
+
+	@Test
+	public void testFloat4() throws Exception {
+		doSingleColumnAOTest("float4");
+	}
+
+	@Test
+	public void testFloat8() throws Exception {
+		doSingleColumnAOTest("float8");
+	}
+
+	@Test
+	public void testNumeric() throws Exception {
+		doSingleColumnAOTest("numeric");
+	}
+
+	@Test
+	public void testCharN() throws Exception {
+		doSingleColumnAOTest("char(10)");
+	}
+
+	@Test
+	public void testVarcharN() throws Exception {
+		doSingleColumnAOTest("varchar(10)");
+	}
+
+	@Test
+	public void testText() throws Exception {
+		doSingleColumnAOTest("text");
+	}
+
+	@Test
+	public void testDate() throws Exception {
+		doSingleColumnAOTest("date");
+	}
+
+	@Test
+	public void testTime() throws Exception {
+		doSingleColumnAOTest("time");
+	}
+
+	@Ignore("we cannot use a static answer file for timetz as we did in all " +
+			"UTs, because its value depends on the machine running the tests." +
+			"Nonetheless, timetz will still be tested in FT")
+	@Test
+	public void testTimetz() throws Exception {
+		doSingleColumnAOTest("timetz");
+	}
+
+	@Test
+	public void testTimestamp() throws Exception {
+		doSingleColumnAOTest("timestamp");
+	}
+
+	@Ignore("the same reason as timetz")
+	@Test
+	public void testTimestamptz() throws Exception {
+		doSingleColumnAOTest("timestamptz");
+	}
+
+	@Test
+	public void testInterval() throws Exception {
+		doSingleColumnAOTest("interval");
+	}
+
+	@Test
+	public void testPoint() throws Exception {
+		doSingleColumnAOTest("point");
+	}
+
+	@Test
+	public void testLseg() throws Exception {
+		doSingleColumnAOTest("lseg");
+	}
+
+	@Test
+	public void testBox() throws Exception {
+		doSingleColumnAOTest("box");
+	}
+
+	@Test
+	public void testCircle() throws Exception {
+		doSingleColumnAOTest("circle");
+	}
+
+	@Test
+	public void testPath() throws Exception {
+		doSingleColumnAOTest("path");
+	}
+
+	@Test
+	public void testPolygon() throws Exception {
+		doSingleColumnAOTest("polygon");
+	}
+
+	@Test
+	public void testMacaddr() throws Exception {
+		doSingleColumnAOTest("macaddr");
+	}
+
+	@Test
+	public void testInet() throws Exception {
+		doSingleColumnAOTest("inet");
+	}
+
+	@Test
+	public void testCidr() throws Exception {
+		doSingleColumnAOTest("cidr");
+	}
+
+	@Test
+	public void testXml() throws Exception {
+		doSingleColumnAOTest("xml");
+	}
+
+	@Test
+	public void testArray() throws Exception {
+		doSingleColumnAOTest("int4[]");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Compression.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Compression.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Compression.java
new file mode 100644
index 0000000..96a2863
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Compression.java
@@ -0,0 +1,44 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.DataProvider;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.RandomDataProvider;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Test reading Parquet compressed table.
+ */
+public class HAWQInputFormatUnitTest_Parquet_Compression extends SimpleTableLocalTester {
+
+	private List<String> colTypes = Lists.newArrayList("int4", "text");
+	private DataProvider provider = new RandomDataProvider(500);
+
+	@Test
+	public void testSnappy() throws Exception {
+		HAWQTable table = new HAWQTable.Builder("test_parquet_snappy", colTypes)
+				.storage(HAWQTableFormat.Parquet)
+				.compress("snappy", 0)
+				.provider(provider)
+				.build();
+
+		testSimpleTable(table);
+	}
+
+	@Test
+	public void testGzip() throws Exception {
+		for (int compressLevel = 1; compressLevel < 10; compressLevel++) {
+			HAWQTable table = new HAWQTable.Builder("test_parquet_gzip_" + compressLevel, colTypes)
+					.storage(HAWQTableFormat.Parquet)
+					.compress("gzip", compressLevel)
+					.provider(provider)
+					.build();
+
+			testSimpleTable(table);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Misc.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Misc.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Misc.java
new file mode 100644
index 0000000..56fe265
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Misc.java
@@ -0,0 +1,34 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.DataProvider;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Test;
+
+/**
+ * Miscellaneous tests for Parquet tables.
+ */
+public class HAWQInputFormatUnitTest_Parquet_Misc extends SimpleTableLocalTester {
+
+	@Test
+	public void testParquetEmptyTable() throws Exception {
+		HAWQTable table = new HAWQTable.Builder("test_parquet_empty", Lists.newArrayList("int4"))
+				.storage(HAWQTableFormat.Parquet)
+				.provider(DataProvider.EMPTY)
+				.build();
+
+		testSimpleTable(table);
+	}
+
+	@Test
+	public void testParquetRecordGetAllTypes() throws Exception {
+		HAWQTable table = new HAWQTable.Builder("test_parquet_alltypes", UnitTestAllTypesMapper.types)
+				.storage(HAWQTableFormat.Parquet)
+				.provider(DataProvider.RANDOM)
+				.build();
+
+		testSimpleTable(table, UnitTestAllTypesMapper.class);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Options.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Options.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Options.java
new file mode 100644
index 0000000..fbda61e
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Options.java
@@ -0,0 +1,50 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.SeriesIntDataProvider;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * Test reading Parquet tables with different options.
+ */
+public class HAWQInputFormatUnitTest_Parquet_Options extends SimpleTableLocalTester {
+	List<String> colTypes = Lists.newArrayList("int8");
+
+	@Test
+	public void testPageSize() throws Exception {
+		int[] pageSizes = {262144, 524288}; // 256K 512K
+
+		for (int pageSize : pageSizes) {
+			String tableName = "test_parquet_pagesize_" + pageSize;
+			HAWQTable table = new HAWQTable.Builder(tableName, colTypes)
+					.storage(HAWQTableFormat.Parquet)
+					.pageSize(pageSize)
+					.provider(new SeriesIntDataProvider(pageSize))
+					.build();
+
+			testSimpleTable(table);
+		}
+	}
+
+	@Test
+	public void testRowGroupSize() throws Exception {
+		int[] rowGroupSizes = {1048576, 2097152};  // 1M 2M
+
+		for (int rowGroupSize : rowGroupSizes) {
+			String tableName = "test_parquet_rowgroupsize_" + rowGroupSize;
+			HAWQTable table = new HAWQTable.Builder(tableName, colTypes)
+					.storage(HAWQTableFormat.Parquet)
+					.rowGroupSize(rowGroupSize)
+					.pageSize(rowGroupSize / 2)
+					.provider(new SeriesIntDataProvider(rowGroupSize / 2))
+					.build();
+
+			testSimpleTable(table);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Types.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Types.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Types.java
new file mode 100644
index 0000000..db9a0c7
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_Parquet_Types.java
@@ -0,0 +1,181 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.HAWQTable;
+import com.pivotal.hawq.mapreduce.SimpleTableLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Ignore;
+import org.junit.Test;
+
+/**
+ * Test data type support for Parquet table.
+ */
+public class HAWQInputFormatUnitTest_Parquet_Types extends SimpleTableLocalTester {
+
+	private void doSingleColumnParquetTest(String columnType) throws Exception {
+		String tableName = "test_parquet_types_" + columnType;
+		tableName = tableName.replace("(", "")
+							 .replace(")", "")
+							 .replace("[]", "_array");
+
+		HAWQTable table = new HAWQTable.Builder(tableName, Lists.newArrayList(columnType))
+				.storage(HAWQTableFormat.Parquet)
+				.build();
+
+		testSimpleTable(table);
+	}
+
+	@Test
+	public void testBool() throws Exception {
+		doSingleColumnParquetTest("bool");
+	}
+
+	@Test
+	public void testBit() throws Exception {
+		doSingleColumnParquetTest("bit");
+	}
+
+	@Test
+	public void testBitN() throws Exception {
+		doSingleColumnParquetTest("bit(5)");
+	}
+
+	@Test
+	public void testVarbit() throws Exception {
+		doSingleColumnParquetTest("varbit");
+	}
+
+	@Test
+	public void testByteArray() throws Exception {
+		doSingleColumnParquetTest("bytea");
+	}
+
+	@Test
+	public void testInt2() throws Exception {
+		doSingleColumnParquetTest("int2");
+	}
+
+	@Test
+	public void testInt4() throws Exception {
+		doSingleColumnParquetTest("int4");
+	}
+
+	@Test
+	public void testInt8() throws Exception {
+		doSingleColumnParquetTest("int8");
+	}
+
+	@Test
+	public void testFloat4() throws Exception {
+		doSingleColumnParquetTest("float4");
+	}
+
+	@Test
+	public void testFloat8() throws Exception {
+		doSingleColumnParquetTest("float8");
+	}
+
+	@Test
+	public void testNumeric() throws Exception {
+		doSingleColumnParquetTest("numeric");
+	}
+
+	@Test
+	public void testCharN() throws Exception {
+		doSingleColumnParquetTest("char(10)");
+	}
+
+	@Test
+	public void testVarcharN() throws Exception {
+		doSingleColumnParquetTest("varchar(10)");
+	}
+
+	@Test
+	public void testText() throws Exception {
+		doSingleColumnParquetTest("text");
+	}
+
+	@Test
+	public void testDate() throws Exception {
+		doSingleColumnParquetTest("date");
+	}
+
+	@Test
+	public void testTime() throws Exception {
+		doSingleColumnParquetTest("time");
+	}
+
+	@Ignore("we cannot use a static answer file for timetz as we did in all " +
+			"UTs, because its value depends on the machine running the tests." +
+			"Nonetheless, timetz will still be tested in FT")
+	@Test
+	public void testTimetz() throws Exception {
+		doSingleColumnParquetTest("timetz");
+	}
+
+	@Test
+	public void testTimestamp() throws Exception {
+		doSingleColumnParquetTest("timestamp");
+	}
+
+	@Ignore("the same reason as timetz")
+	@Test
+	public void testTimestamptz() throws Exception {
+		doSingleColumnParquetTest("timestamptz");
+	}
+
+	@Test
+	public void testInterval() throws Exception {
+		doSingleColumnParquetTest("interval");
+	}
+
+	@Test
+	public void testPoint() throws Exception {
+		doSingleColumnParquetTest("point");
+	}
+
+	@Test
+	public void testLseg() throws Exception {
+		doSingleColumnParquetTest("lseg");
+	}
+
+	@Test
+	public void testBox() throws Exception {
+		doSingleColumnParquetTest("box");
+	}
+
+	@Test
+	public void testCircle() throws Exception {
+		doSingleColumnParquetTest("circle");
+	}
+
+	@Test
+	public void testPath() throws Exception {
+		doSingleColumnParquetTest("path");
+	}
+
+	@Test
+	public void testPolygon() throws Exception {
+		doSingleColumnParquetTest("polygon");
+	}
+
+	@Test
+	public void testMacaddr() throws Exception {
+		doSingleColumnParquetTest("macaddr");
+	}
+
+	@Test
+	public void testInet() throws Exception {
+		doSingleColumnParquetTest("inet");
+	}
+
+	@Test
+	public void testCidr() throws Exception {
+		doSingleColumnParquetTest("cidr");
+	}
+
+	@Test
+	public void testXml() throws Exception {
+		doSingleColumnParquetTest("xml");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_TPCH.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_TPCH.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_TPCH.java
new file mode 100644
index 0000000..a72755d
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/HAWQInputFormatUnitTest_TPCH.java
@@ -0,0 +1,35 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.pivotal.hawq.mapreduce.TPCHLocalTester;
+import com.pivotal.hawq.mapreduce.metadata.HAWQTableFormat;
+import org.junit.Test;
+
+/**
+ * Test reading TPC-H table using HAWQInputFormat
+ */
+public class HAWQInputFormatUnitTest_TPCH extends TPCHLocalTester {
+
+	@Test
+	public void testTPCH_AO_No_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.001", HAWQTableFormat.AO, false);
+		testTPCHTable(tpchSpec, "lineitem_ao_row");
+	}
+
+	@Test
+	public void testTPCH_AO_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.001", HAWQTableFormat.AO, true);
+		testTPCHTable(tpchSpec, "lineitem_ao_row");
+	}
+
+	@Test
+	public void testTPCH_Parquet_No_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.001", HAWQTableFormat.Parquet, false);
+		testTPCHTable(tpchSpec, "lineitem_ao_parquet");
+	}
+
+	@Test
+	public void testTPCH_Parquet_Partition() throws Exception {
+		HAWQTPCHSpec tpchSpec = new HAWQTPCHSpec("0.001", HAWQTableFormat.Parquet, true);
+		testTPCHTable(tpchSpec, "lineitem_ao_parquet");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/UnitTestAllTypesMapper.java
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/UnitTestAllTypesMapper.java b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/UnitTestAllTypesMapper.java
new file mode 100644
index 0000000..a4767ff
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/java/com/pivotal/hawq/mapreduce/ut/UnitTestAllTypesMapper.java
@@ -0,0 +1,80 @@
+package com.pivotal.hawq.mapreduce.ut;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.pivotal.hawq.mapreduce.HAWQException;
+import com.pivotal.hawq.mapreduce.HAWQRecord;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Mapper;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Mapper class used in test_ao_alltypes and test_parquet_alltypes UT.
+ */
+class UnitTestAllTypesMapper extends Mapper<Void, HAWQRecord, Text, Text> {
+	private String fieldToString(Object val) throws HAWQException {
+		if (val == null) return "null";
+		if (val instanceof byte[]) return new String((byte[]) val);
+		return val.toString();
+	}
+
+	static final String SEPARATOR = "|";
+
+	// Due to floating point values are tricker to test equality, we only allow float4/float8
+	// in single column test table. Therefore we don't include float4/float8 in all type tests.
+	// timetz/timestamptz is also excluded due to timezone issue.
+	static final List<String> types = Lists.newArrayList(
+			"bool", "bit", "varbit", "bytea", "int2", "int4", "int8", /*"float4", "float8",*/ "numeric",
+			"char(10)", "varchar(10)", "text", "date", "time", /*"timetz",*/ "timestamp", /*"timestamptz",*/ "interval",
+			"point", "lseg", "box", "circle", "path", "polygon", "macaddr", "inet", "cidr", "xml");
+
+	@Override
+	protected void map(Void key, HAWQRecord value, Context context)
+			throws IOException, InterruptedException {
+		try {
+			List<String> values = Lists.newArrayList();
+
+			values.add(fieldToString(value.getBoolean(1)));
+			values.add(fieldToString(value.getByte(2)));
+			values.add(fieldToString(value.getVarbit(3)));
+			values.add(fieldToString(value.getBytes(4)));
+			values.add(fieldToString(value.getShort(5)));
+			values.add(fieldToString(value.getInt(6)));
+			values.add(fieldToString(value.getLong(7)));
+			values.add(fieldToString(value.getBigDecimal(8)));
+			values.add(fieldToString(value.getString(9)));
+			values.add(fieldToString(value.getString(10)));
+			values.add(fieldToString(value.getString(11)));
+			values.add(fieldToString(value.getDate(12)));
+			values.add(fieldToString(value.getTime(13)));
+			values.add(fieldToString(value.getTimestamp(14)));
+			values.add(fieldToString(value.getInterval(15)));
+			values.add(fieldToString(value.getPoint(16)));
+			values.add(fieldToString(value.getLseg(17)));
+			values.add(fieldToString(value.getBox(18)));
+			values.add(fieldToString(value.getCircle(19)));
+			values.add(fieldToString(value.getPath(20)));
+			values.add(fieldToString(value.getPolygon(21)));
+			values.add(fieldToString(value.getMacaddr(22)));
+			values.add(fieldToString(value.getInet(23)));
+			values.add(fieldToString(value.getCidr(24)));
+			values.add(fieldToString(value.getString(25)));
+
+			// some type's get method will return a default value instead of null,
+			// we must take care of it!
+			for (int i  = 1; i <= value.getSchema().getFieldCount(); i++) {
+				if (value.isNull(i)) {
+					values.set(i - 1, "null");
+				}
+			}
+
+			Text text = new Text(Joiner.on(SEPARATOR).join(values));
+			context.write(text, text);
+
+		} catch (HAWQException e) {
+			throw new IOException(e);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/dataset
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/dataset b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/dataset
new file mode 100644
index 0000000..e376973
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/dataset
@@ -0,0 +1,31 @@
+bool  #  'true'  'false'  null
+bit  #  '1'  '0'  null
+bit(5)  #  '10010'  '00010'  null
+varbit  #  '1111'  '000000000011100'  null
+bytea  #  'hello world'  'aaaaaaaaaaaaaaaaaaa'  null
+int2  #  -32768  -128  0  128  32767  null
+int4  #  -2147483648  -32768  -128  0  128  32767  2147483647  null
+int8  #  -9223372036854775808  -2147483648  -32768  -128  0  128  32767  2147483647  9223372036854775807  null
+float4  #  -12.3456  0  1.00  2.001  9999.1234  123456  'Infinity'  '-Infinity'  'NaN'  null
+float8  #  -123.456  -12.8765  0  1.00  2.001  128  1298765  999999  12345  'Infinity'  '-Infinity'  'NaN'  null
+numeric  #  6.54  0.0001  null
+char(10)  #  '123456789a'  'bbccddeeff'  'aaaa'  null
+varchar(10)  #  '123456789a'  'bbccddeeff'  'aaaa'  null
+text  #  ''  'z'  'hello world'  repeat('b',1000)  null
+date  #  '4713-01-01 BC'  '2014-03-02'  '4277-12-31 AD'  null
+time  #  '00:00:00'  '04:05:06'  '23:59:59'  null
+timetz  #  '12:03:00+00'  '23:50:10-08'  '04:45:05.0012+08:40'  null
+timestamp  #  '4713-01-01 BC'  '2942-12-31 AD'  '1999-01-08 04:05:06'  '2004-10-19 10:23:54'  null
+timestamptz  #  '1999-12-31 23:00:00+08'  '2000-01-01 08:00:00+09'  null
+interval  #  '-178000000 years'  '178000000 years'  null
+point  #  POINT(1,2)  POINT(100,200)  POINT(1000,2000)  null
+lseg  #  '[(0,0),(6,6)]'  '[(1,1),(2,2)]'  null
+box  #  '((0,1),(2,3))'  '((100,200),(200,400))'  null
+circle  #  '<(1,2),3>'  '<(100,200),300>'  null
+path  #  '(1,1)'  '(1,1),(2,3),(4,5)'  null
+polygon  #  '((1,1),(3,2),(4,5))'  '((100,123),(5,10),(7,2),(4,5))'  null
+macaddr  #  'FF:89:71:45:AE:01'  null
+inet  #  '2001:db8:85a3:8d3:1319:8a2e:370:7344/64'  '2001::7344'  '172.20.143.0'  '192.168.1.255/24'  null
+cidr  #  '2001:db8:85a3:8d3:1319:8a2e:370:7344/128'  '192.168.1.255/32'  '172.20.143.0/24'  null
+xml  #  '<aa>bb</aa>'  '<name>wj</name>'  null
+int4[]  #  '{1,2,3,4}'  '{{1,2},{3,4}}'  '{{{1,2},{3,4}},{{5,6},{7,8}}}'  null
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/log4j.properties b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/log4j.properties
new file mode 100644
index 0000000..583881f
--- /dev/null
+++ b/contrib/hawq-hadoop/hawq-mapreduce-tool/src/test/resources/log4j.properties
@@ -0,0 +1,231 @@
+# Copyright 2011 The Apache Software Foundation
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# Define some default values that can be overridden by system properties
+hadoop.root.logger=INFO,console
+hadoop.log.dir=.
+hadoop.log.file=hadoop.log
+
+# Define the root logger to the system property "hadoop.root.logger".
+log4j.rootLogger=${hadoop.root.logger}, EventCounter
+
+# Logging Threshold
+log4j.threshold=ALL
+
+# Null Appender
+log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
+
+#
+# Rolling File Appender - cap space usage at 5gb.
+#
+hadoop.log.maxfilesize=256MB
+hadoop.log.maxbackupindex=20
+log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+log4j.appender.RFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+log4j.appender.RFA.MaxFileSize=${hadoop.log.maxfilesize}
+log4j.appender.RFA.MaxBackupIndex=${hadoop.log.maxbackupindex}
+
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+# Debugging Pattern format
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+
+#
+# TaskLog Appender
+#
+
+#Default values
+hadoop.tasklog.taskid=null
+hadoop.tasklog.iscleanup=false
+hadoop.tasklog.noKeepSplits=4
+hadoop.tasklog.totalLogFileSize=100
+hadoop.tasklog.purgeLogSplits=true
+hadoop.tasklog.logsRetainHours=12
+
+log4j.appender.TLA=org.apache.hadoop.mapred.TaskLogAppender
+log4j.appender.TLA.taskId=${hadoop.tasklog.taskid}
+log4j.appender.TLA.isCleanup=${hadoop.tasklog.iscleanup}
+log4j.appender.TLA.totalLogFileSize=${hadoop.tasklog.totalLogFileSize}
+
+log4j.appender.TLA.layout=org.apache.log4j.PatternLayout
+log4j.appender.TLA.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+
+#
+# HDFS block state change log from block manager
+#
+# Uncomment the following to suppress normal block state change
+# messages from BlockManager in NameNode.
+#log4j.logger.BlockStateChange=WARN
+
+#
+#Security appender
+#
+hadoop.security.logger=INFO,NullAppender
+hadoop.security.log.maxfilesize=256MB
+hadoop.security.log.maxbackupindex=20
+log4j.category.SecurityLogger=${hadoop.security.logger}
+hadoop.security.log.file=SecurityAuth-${user.name}.audit
+log4j.appender.RFAS=org.apache.log4j.RollingFileAppender
+log4j.appender.RFAS.File=${hadoop.log.dir}/${hadoop.security.log.file}
+log4j.appender.RFAS.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.RFAS.MaxFileSize=${hadoop.security.log.maxfilesize}
+log4j.appender.RFAS.MaxBackupIndex=${hadoop.security.log.maxbackupindex}
+
+#
+# Daily Rolling Security appender
+#
+log4j.appender.DRFAS=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFAS.File=${hadoop.log.dir}/${hadoop.security.log.file}
+log4j.appender.DRFAS.layout=org.apache.log4j.PatternLayout
+log4j.appender.DRFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.appender.DRFAS.DatePattern=.yyyy-MM-dd
+
+#
+# hadoop configuration logging
+#
+
+# Uncomment the following line to turn off configuration deprecation warnings.
+# log4j.logger.org.apache.hadoop.conf.Configuration.deprecation=WARN
+
+#
+# hdfs audit logging
+#
+hdfs.audit.logger=INFO,NullAppender
+hdfs.audit.log.maxfilesize=256MB
+hdfs.audit.log.maxbackupindex=20
+log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=${hdfs.audit.logger}
+log4j.additivity.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=false
+log4j.appender.RFAAUDIT=org.apache.log4j.RollingFileAppender
+log4j.appender.RFAAUDIT.File=${hadoop.log.dir}/hdfs-audit.log
+log4j.appender.RFAAUDIT.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFAAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+log4j.appender.RFAAUDIT.MaxFileSize=${hdfs.audit.log.maxfilesize}
+log4j.appender.RFAAUDIT.MaxBackupIndex=${hdfs.audit.log.maxbackupindex}
+
+#
+# mapred audit logging
+#
+mapred.audit.logger=INFO,NullAppender
+mapred.audit.log.maxfilesize=256MB
+mapred.audit.log.maxbackupindex=20
+log4j.logger.org.apache.hadoop.mapred.AuditLogger=${mapred.audit.logger}
+log4j.additivity.org.apache.hadoop.mapred.AuditLogger=false
+log4j.appender.MRAUDIT=org.apache.log4j.RollingFileAppender
+log4j.appender.MRAUDIT.File=${hadoop.log.dir}/mapred-audit.log
+log4j.appender.MRAUDIT.layout=org.apache.log4j.PatternLayout
+log4j.appender.MRAUDIT.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n
+log4j.appender.MRAUDIT.MaxFileSize=${mapred.audit.log.maxfilesize}
+log4j.appender.MRAUDIT.MaxBackupIndex=${mapred.audit.log.maxbackupindex}
+
+# Custom Logging levels
+
+#log4j.logger.org.apache.hadoop.mapred.JobTracker=DEBUG
+#log4j.logger.org.apache.hadoop.mapred.TaskTracker=DEBUG
+#log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=DEBUG
+
+# Jets3t library
+log4j.logger.org.jets3t.service.impl.rest.httpclient.RestS3Service=ERROR
+
+#
+# Event Counter Appender
+# Sends counts of logging messages at different severity levels to Hadoop Metrics.
+#
+log4j.appender.EventCounter=org.apache.hadoop.log.metrics.EventCounter
+
+#
+# Job Summary Appender
+#
+# Use following logger to send summary to separate file defined by
+# hadoop.mapreduce.jobsummary.log.file :
+# hadoop.mapreduce.jobsummary.logger=INFO,JSA
+#
+hadoop.mapreduce.jobsummary.logger=${hadoop.root.logger}
+hadoop.mapreduce.jobsummary.log.file=hadoop-mapreduce.jobsummary.log
+hadoop.mapreduce.jobsummary.log.maxfilesize=256MB
+hadoop.mapreduce.jobsummary.log.maxbackupindex=20
+log4j.appender.JSA=org.apache.log4j.RollingFileAppender
+log4j.appender.JSA.File=${hadoop.log.dir}/${hadoop.mapreduce.jobsummary.log.file}
+log4j.appender.JSA.MaxFileSize=${hadoop.mapreduce.jobsummary.log.maxfilesize}
+log4j.appender.JSA.MaxBackupIndex=${hadoop.mapreduce.jobsummary.log.maxbackupindex}
+log4j.appender.JSA.layout=org.apache.log4j.PatternLayout
+log4j.appender.JSA.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{2}: %m%n
+log4j.logger.org.apache.hadoop.mapred.JobInProgress$JobSummary=${hadoop.mapreduce.jobsummary.logger}
+log4j.additivity.org.apache.hadoop.mapred.JobInProgress$JobSummary=false
+
+#
+# Yarn ResourceManager Application Summary Log
+#
+# Set the ResourceManager summary log filename
+yarn.server.resourcemanager.appsummary.log.file=rm-appsummary.log
+# Set the ResourceManager summary log level and appender
+yarn.server.resourcemanager.appsummary.logger=${hadoop.root.logger}
+#yarn.server.resourcemanager.appsummary.logger=INFO,RMSUMMARY
+
+# To enable AppSummaryLogging for the RM,
+# set yarn.server.resourcemanager.appsummary.logger to
+# <LEVEL>,RMSUMMARY in hadoop-env.sh
+
+# Appender for ResourceManager Application Summary Log
+# Requires the following properties to be set
+#    - hadoop.log.dir (Hadoop Log directory)
+#    - yarn.server.resourcemanager.appsummary.log.file (resource manager app summary log filename)
+#    - yarn.server.resourcemanager.appsummary.logger (resource manager app summary log level and appender)
+
+log4j.logger.org.apache.hadoop.yarn.server.resourcemanager.RMAppManager$ApplicationSummary=${yarn.server.resourcemanager.appsummary.logger}
+log4j.additivity.org.apache.hadoop.yarn.server.resourcemanager.RMAppManager$ApplicationSummary=false
+log4j.appender.RMSUMMARY=org.apache.log4j.RollingFileAppender
+log4j.appender.RMSUMMARY.File=${hadoop.log.dir}/${yarn.server.resourcemanager.appsummary.log.file}
+log4j.appender.RMSUMMARY.MaxFileSize=256MB
+log4j.appender.RMSUMMARY.MaxBackupIndex=20
+log4j.appender.RMSUMMARY.layout=org.apache.log4j.PatternLayout
+log4j.appender.RMSUMMARY.layout.ConversionPattern=%d{ISO8601} %p %c{2}: %m%n

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/16555/16601.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/16555/16601.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/16555/16601.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179546.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179546.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179546.1
new file mode 100644
index 0000000..2628b4a
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179546.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179551.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179551.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179551.1
new file mode 100644
index 0000000..b612e3d
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179551.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179556.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179556.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179556.1
new file mode 100644
index 0000000..ff832ab
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179556.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179561.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179561.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179561.1
new file mode 100644
index 0000000..357491e
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179561.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179566.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179566.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179566.1
new file mode 100644
index 0000000..a2c6e3b
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179566.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179571.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179571.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179571.1
new file mode 100644
index 0000000..b3457e8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179571.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179576.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179576.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179576.1
new file mode 100644
index 0000000..bec9d09
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179576.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179581.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179581.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179581.1
new file mode 100644
index 0000000..49fe178
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179581.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179586.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179586.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179586.1
new file mode 100644
index 0000000..3bc150b
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179586.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179591.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179591.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179591.1
new file mode 100644
index 0000000..ece23ee
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179591.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179596.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179596.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179596.1
new file mode 100644
index 0000000..d4d5b6c
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179596.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179601.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179601.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179601.1
new file mode 100644
index 0000000..6e5121e
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179601.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179606.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179606.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179606.1
new file mode 100644
index 0000000..0199e14
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179606.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179626.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179626.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179626.1
new file mode 100644
index 0000000..3718930
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179626.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179631.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179631.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179631.1
new file mode 100644
index 0000000..f084c01
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179631.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179636.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179636.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179636.1
new file mode 100644
index 0000000..0aa9a7c
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179636.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179641.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179641.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179641.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179641.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179646.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179646.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179646.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179646.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179651.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179651.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179651.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179651.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179656.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179656.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179656.1
new file mode 100644
index 0000000..7b1f598
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179656.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179661.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179661.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179661.1
new file mode 100644
index 0000000..959e6ff
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179661.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179666.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179666.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179666.1
new file mode 100644
index 0000000..1c3be8a
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179666.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179671.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179671.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179671.1
new file mode 100644
index 0000000..7061117
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179671.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179676.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179676.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179676.1
new file mode 100644
index 0000000..c4445f2
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179676.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179681.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179681.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179681.1
new file mode 100644
index 0000000..3232b18
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179681.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179686.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179686.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179686.1
new file mode 100644
index 0000000..17bd357
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179686.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179691.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179691.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179691.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179691.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179696.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179696.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179696.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179696.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179701.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179701.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179701.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179701.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179706.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179706.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179706.1
new file mode 100644
index 0000000..2a8d424
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179706.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179711.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179711.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179711.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179711.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179721.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179721.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179721.1
new file mode 100644
index 0000000..22da96f
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179721.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179731.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179731.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179731.1
new file mode 100644
index 0000000..1beeeb8
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179731.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179736.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179736.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179736.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179741.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179741.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179741.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179746.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179746.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179746.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179751.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179751.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179751.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179756.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179756.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179756.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179761.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179761.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179761.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179766.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179766.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179766.1
new file mode 100644
index 0000000..41f1b92
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179766.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179771.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179771.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179771.1
new file mode 100644
index 0000000..9d18de7
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179771.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179776.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179776.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179776.1
new file mode 100644
index 0000000..be5772f
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179776.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179781.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179781.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179781.1
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179786.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179786.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179786.1
new file mode 100644
index 0000000..3c22623
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179786.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179801.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179801.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179801.1
new file mode 100644
index 0000000..80ada0c
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179801.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179806.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179806.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179806.1
new file mode 100644
index 0000000..f9793ea
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179806.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179811.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179811.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179811.1
new file mode 100644
index 0000000..d2b4364
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179811.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179816.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179816.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179816.1
new file mode 100644
index 0000000..b3d1cde
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179816.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179821.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179821.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179821.1
new file mode 100644
index 0000000..36d4b33
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179821.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179826.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179826.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179826.1
new file mode 100644
index 0000000..3db9971
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179826.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179831.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179831.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179831.1
new file mode 100644
index 0000000..9150a04
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179831.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179836.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179836.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179836.1
new file mode 100644
index 0000000..d56a5c7
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179836.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179841.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179841.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179841.1
new file mode 100644
index 0000000..e99e2f1
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179841.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179846.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179846.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179846.1
new file mode 100644
index 0000000..aa8d4cc
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179846.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179851.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179851.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179851.1
new file mode 100644
index 0000000..92205f4
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179851.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179871.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179871.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179871.1
new file mode 100644
index 0000000..e65f784
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179871.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179891.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179891.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179891.1
new file mode 100644
index 0000000..0b66c88
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179891.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179911.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179911.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179911.1
new file mode 100644
index 0000000..20683f5
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179911.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179931.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179931.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179931.1
new file mode 100644
index 0000000..a889476
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179931.1 differ

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/8b26974c/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179936.1
----------------------------------------------------------------------
diff --git a/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179936.1 b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179936.1
new file mode 100644
index 0000000..66136fc
Binary files /dev/null and b/contrib/hawq-hadoop/hawq-mapreduce-tool/test-data/hawq-data/gpseg0/16385/179545/179936.1 differ