You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by bl...@apache.org on 2014/03/06 06:49:27 UTC

git commit: TAJO-427: Empty table makes IndexOutOfBoundsException at LEFT OUTER JOIN clause. (jaehwa)

Repository: incubator-tajo
Updated Branches:
  refs/heads/master 462abc471 -> 4089c9d59


TAJO-427: Empty table makes IndexOutOfBoundsException at LEFT OUTER JOIN clause. (jaehwa)


Project: http://git-wip-us.apache.org/repos/asf/incubator-tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tajo/commit/4089c9d5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/4089c9d5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/4089c9d5

Branch: refs/heads/master
Commit: 4089c9d59d1fd150430746caa4a4031ba3dd8279
Parents: 462abc4
Author: blrunner <jh...@gruter.com>
Authored: Thu Mar 6 14:49:07 2014 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Thu Mar 6 14:49:07 2014 +0900

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../planner/logical/ShuffleFileWriteNode.java   |  4 +-
 .../planner/physical/HashPartitioner.java       |  5 ++
 .../engine/planner/physical/Partitioner.java    |  4 +-
 .../org/apache/tajo/engine/utils/TupleUtil.java | 26 +++++++++--
 .../apache/tajo/engine/query/TestJoinQuery.java | 49 ++++++++++++++++++++
 .../testCrossJoinWithEmptyTable1.sql            |  8 ++++
 .../testFullOuterJoinWithEmptyTable1.sql        |  8 ++++
 .../TestJoinQuery/testLeftOuterJoin1.sql        |  4 +-
 .../testLeftOuterJoinWithEmptyTable1.sql        |  9 ++++
 .../testLeftOuterJoinWithEmptyTable2.sql        |  9 ++++
 .../testLeftOuterJoinWithEmptyTable3.sql        | 14 ++++++
 .../testLeftOuterJoinWithEmptyTable4.sql        | 17 +++++++
 .../testRightOuterJoinWithEmptyTable1.sql       |  8 ++++
 .../testCrossJoinWithEmptyTable1.result         |  2 +
 .../testFullOuterJoinWithEmptyTable1.result     |  7 +++
 .../TestJoinQuery/testLeftOuterJoin1.result     | 12 ++---
 .../testLeftOuterJoinWithEmptyTable1.result     |  7 +++
 .../testLeftOuterJoinWithEmptyTable2.result     |  7 +++
 .../testLeftOuterJoinWithEmptyTable3.result     |  7 +++
 .../testLeftOuterJoinWithEmptyTable4.result     |  4 ++
 .../testRightOuterJoinWithEmptyTable1.result    |  7 +++
 22 files changed, 207 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d7f5882..52f4543 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -269,6 +269,8 @@ Release 0.8.0 - unreleased
 
   BUG FIXES
 
+    TAJO-427: Empty table makes IndexOutOfBoundsException at LEFT OUTER JOIN clause. (jaehwa)
+
     TAJO-650: Repartitioner::scheduleHashShuffledFetches should adjust the
     number of tasks. (hyunsik)
 

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java
index 5399357..c7ea454 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ShuffleFileWriteNode.java
@@ -53,7 +53,9 @@ public class ShuffleFileWriteNode extends PersistentStoreNode implements Cloneab
   public final void setShuffle(TajoWorkerProtocol.ShuffleType type, Column[] keys, int numPartitions) {
     Preconditions.checkArgument(keys.length >= 0, 
         "At least one partition key must be specified.");
-    Preconditions.checkArgument(numPartitions > 0,
+    // In outer join, zero can be passed into this value because of empty tables.
+    // So, we should allow zero.
+    Preconditions.checkArgument(numPartitions >= 0,
         "The number of partitions must be positive: %s", numPartitions);
 
     this.shuffleType = type;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java
index b620b22..3ae53d9 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/HashPartitioner.java
@@ -34,6 +34,11 @@ public class HashPartitioner extends Partitioner {
   
   @Override
   public int getPartition(Tuple tuple) {
+    // In outer join, partition number can be zero because of empty tables.
+    // So, we should return zero for this case.
+    if (numPartitions == 0)
+      return 0;
+
     // build one key tuple
     for (int i = 0; i < partitionKeyIds.length; i++) {
       keyTuple.put(i, tuple.get(partitionKeyIds[i]));

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java
index b67f45c..6d544a7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/Partitioner.java
@@ -33,7 +33,9 @@ public abstract class Partitioner {
         "Partition keys must be given");
     Preconditions.checkArgument(keyList.length >= 0,
         "At least one partition key must be specified.");
-    Preconditions.checkArgument(numPartitions > 0, 
+    // In outer join, zero can be passed into this value because of empty tables.
+    // So, we should allow zero.
+    Preconditions.checkArgument(numPartitions >= 0,
         "The number of partitions must be positive: %s", numPartitions);
     this.partitionKeyIds = keyList;
     this.numPartitions = numPartitions;    

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
index 6d801dd..54c6f74 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/utils/TupleUtil.java
@@ -83,13 +83,31 @@ public class TupleUtil {
     Tuple startTuple = new VTuple(target.size());
     Tuple endTuple = new VTuple(target.size());
     int i = 0;
+
+    // In outer join, empty table could be searched.
+    // As a result, min value and max value would be null.
+    // So, we should put NullDatum for this case.
     for (Column col : target.getColumns()) {
       if (sortSpecs[i].isAscending()) {
-        startTuple.put(i, statSet.get(col).getMinValue());
-        endTuple.put(i, statSet.get(col).getMaxValue());
+        if (statSet.get(col).getMinValue() != null)
+          startTuple.put(i, statSet.get(col).getMinValue());
+        else
+          startTuple.put(i, DatumFactory.createNullDatum());
+
+        if (statSet.get(col).getMaxValue() != null)
+          endTuple.put(i, statSet.get(col).getMaxValue());
+        else
+          endTuple.put(i, DatumFactory.createNullDatum());
       } else {
-        startTuple.put(i, statSet.get(col).getMaxValue());
-        endTuple.put(i, statSet.get(col).getMinValue());
+        if (statSet.get(col).getMaxValue() != null)
+          startTuple.put(i, statSet.get(col).getMaxValue());
+        else
+          startTuple.put(i, DatumFactory.createNullDatum());
+
+        if (statSet.get(col).getMinValue() != null)
+          endTuple.put(i, statSet.get(col).getMinValue());
+        else
+          endTuple.put(i, DatumFactory.createNullDatum());
       }
       i++;
     }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
index 0e925f1..0c68fa6 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java
@@ -215,4 +215,53 @@ public class TestJoinQuery extends QueryTestCaseBase {
     cleanupQuery(res);
   }
 
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable2() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable3() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testLeftOuterJoinWithEmptyTable4() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testRightOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testFullOuterJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
+  @Test
+  public final void testCrossJoinWithEmptyTable1() throws Exception {
+    ResultSet res = executeQuery();
+    assertResultSet(res);
+    cleanupQuery(res);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithEmptyTable1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithEmptyTable1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithEmptyTable1.sql
new file mode 100644
index 0000000..1e0cb56
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testCrossJoinWithEmptyTable1.sql
@@ -0,0 +1,8 @@
+select
+  c_custkey,
+  empty_orders.o_orderkey
+from
+  customer, empty_orders
+where c_custkey = o_orderkey
+order by
+  c_custkey, o_orderkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testFullOuterJoinWithEmptyTable1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testFullOuterJoinWithEmptyTable1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testFullOuterJoinWithEmptyTable1.sql
new file mode 100644
index 0000000..b9e3efd
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testFullOuterJoinWithEmptyTable1.sql
@@ -0,0 +1,8 @@
+select
+  c_custkey,
+  empty_orders.o_orderkey
+from
+  empty_orders full outer join customer on c_custkey = o_orderkey
+order by
+  c_custkey,
+  empty_orders.o_orderkey;

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoin1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoin1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoin1.sql
index f946e1d..08a68e8 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoin1.sql
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoin1.sql
@@ -1,6 +1,8 @@
 select
   c_custkey,
-  orders.o_orderkey
+  orders.o_orderkey,
+  orders.o_orderstatus,
+  orders.o_orderdate
 from
   customer left outer join orders on c_custkey = o_orderkey
 order by

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.sql
new file mode 100644
index 0000000..27ac8bc
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.sql
@@ -0,0 +1,9 @@
+select
+  c_custkey,
+  empty_orders.o_orderkey,
+  empty_orders.o_orderstatus,
+  empty_orders.o_orderdate
+from
+  customer left outer join empty_orders on c_custkey = o_orderkey
+order by
+  c_custkey, o_orderkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.sql
new file mode 100644
index 0000000..b0639ff
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.sql
@@ -0,0 +1,9 @@
+select
+  c_custkey,
+  sum(empty_orders.o_orderkey),
+  max(empty_orders.o_orderstatus),
+  max(empty_orders.o_orderdate)
+from
+  customer left outer join empty_orders on c_custkey = o_orderkey
+  group by c_custkey
+order by c_custkey ;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.sql
new file mode 100644
index 0000000..6a79e34
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.sql
@@ -0,0 +1,14 @@
+select count(*)
+from (
+  select
+    c_custkey,
+    sum(empty_orders.o_orderkey) as total1,
+    max(empty_orders.o_orderstatus) as total2,
+    max(empty_orders.o_orderdate) as total3
+  from
+    customer left outer join empty_orders on c_custkey = o_orderkey
+    group by c_custkey
+) t1
+group by
+  c_custkey
+order by c_custkey  ;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.sql
new file mode 100644
index 0000000..5f91a18
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.sql
@@ -0,0 +1,17 @@
+
+  select
+    max(c_custkey),
+    sum(orders.o_orderkey),
+    max(orders.o_orderstatus),
+    max(orders.o_orderdate)
+  from
+    customer left outer join orders on c_custkey = o_orderkey
+  union
+  select
+    max(c_custkey),
+    sum(empty_orders.o_orderkey),
+    max(empty_orders.o_orderstatus),
+    max(empty_orders.o_orderdate)
+  from
+    customer left outer join empty_orders on c_custkey = o_orderkey
+;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testRightOuterJoinWithEmptyTable1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testRightOuterJoinWithEmptyTable1.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testRightOuterJoinWithEmptyTable1.sql
new file mode 100644
index 0000000..651a3cf
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestJoinQuery/testRightOuterJoinWithEmptyTable1.sql
@@ -0,0 +1,8 @@
+select
+  c_custkey,
+  empty_orders.o_orderkey
+from
+  empty_orders right outer join customer on c_custkey = o_orderkey
+order by
+  c_custkey,
+  empty_orders.o_orderkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithEmptyTable1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithEmptyTable1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithEmptyTable1.result
new file mode 100644
index 0000000..6f07515
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testCrossJoinWithEmptyTable1.result
@@ -0,0 +1,2 @@
+c_custkey,o_orderkey
+-------------------------------
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testFullOuterJoinWithEmptyTable1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testFullOuterJoinWithEmptyTable1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testFullOuterJoinWithEmptyTable1.result
new file mode 100644
index 0000000..9124c6c
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testFullOuterJoinWithEmptyTable1.result
@@ -0,0 +1,7 @@
+c_custkey,o_orderkey
+-------------------------------
+1,0
+2,0
+3,0
+4,0
+5,0
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoin1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoin1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoin1.result
index 81dc055..8893e12 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoin1.result
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoin1.result
@@ -1,7 +1,7 @@
-c_custkey,o_orderkey
+c_custkey,o_orderkey,o_orderstatus,o_orderdate
 -------------------------------
-1,1
-2,2
-3,3
-4,0
-5,0
\ No newline at end of file
+1,1,O,1996-01-02
+2,2,O,1996-12-01
+3,3,F,1993-10-14
+4,0,,
+5,0,,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.result
new file mode 100644
index 0000000..5e85b28
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable1.result
@@ -0,0 +1,7 @@
+c_custkey,o_orderkey,o_orderstatus,o_orderdate
+-------------------------------
+1,0,,
+2,0,,
+3,0,,
+4,0,,
+5,0,,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result
new file mode 100644
index 0000000..3bf0ed2
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable2.result
@@ -0,0 +1,7 @@
+c_custkey,?sum,?max_1,?max_2
+-------------------------------
+1,0,,
+2,0,,
+3,0,,
+4,0,,
+5,0,,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.result
new file mode 100644
index 0000000..f3e11cb
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable3.result
@@ -0,0 +1,7 @@
+?count
+-------------------------------
+1
+1
+1
+1
+1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result
new file mode 100644
index 0000000..b9ac208
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testLeftOuterJoinWithEmptyTable4.result
@@ -0,0 +1,4 @@
+?max,?sum_1,?max_2,?max_3
+-------------------------------
+5,6,O,1996-12-01
+5,0,,
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/4089c9d5/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testRightOuterJoinWithEmptyTable1.result
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testRightOuterJoinWithEmptyTable1.result b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testRightOuterJoinWithEmptyTable1.result
new file mode 100644
index 0000000..9124c6c
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/resources/results/TestJoinQuery/testRightOuterJoinWithEmptyTable1.result
@@ -0,0 +1,7 @@
+c_custkey,o_orderkey
+-------------------------------
+1,0
+2,0
+3,0
+4,0
+5,0
\ No newline at end of file