You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tajo.apache.org by hy...@apache.org on 2013/09/16 13:33:28 UTC
[1/7] TAJO-184: Refactor GlobalPlanner and global plan data
structure. (hyunsik)
Updated Branches:
refs/heads/master 53df7f455 -> 1b1d1e8c1
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
index 843df23..532bb81 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestRepartitioner.java
@@ -21,7 +21,7 @@ package org.apache.tajo.master;
import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryId;
import org.apache.tajo.TestTajoIds;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
import org.apache.tajo.master.querymaster.QueryUnit;
import org.apache.tajo.master.querymaster.Repartitioner;
import org.apache.tajo.util.TUtil;
@@ -49,7 +49,7 @@ public class TestRepartitioner {
Collection<URI> uris = Repartitioner.
createHashFetchURL(hostName + ":" + port, sid, partitionId,
- PartitionType.HASH, intermediateEntries);
+ TajoWorkerProtocol.PartitionType.HASH_PARTITION, intermediateEntries);
List<String> taList = TUtil.newList();
for (URI uri : uris) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
index 4f2795b..7d93bdf 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/TestRangeRetrieverHandler.java
@@ -23,6 +23,7 @@ import com.google.common.collect.Maps;
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -42,7 +43,6 @@ import org.apache.tajo.engine.planner.physical.ProjectionExec;
import org.apache.tajo.storage.*;
import org.apache.tajo.storage.index.bst.BSTIndex;
import org.apache.tajo.util.CommonTestingUtil;
-import org.apache.tajo.util.TUtil;
import org.apache.tajo.worker.dataserver.retriever.FileChunk;
import org.junit.After;
import org.junit.Before;
@@ -134,7 +134,7 @@ public class TestRangeRetrieverHandler {
Fragment[] frags = StorageManager.splitNG(conf, "employee", employeeMeta, tableDir, Integer.MAX_VALUE);
TaskAttemptContext
- ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(),
new Fragment[] {frags[0]}, testDir);
Expr expr = analyzer.parse(SORT_QUERY[0]);
LogicalPlan plan = planner.createPlan(expr);
@@ -247,7 +247,7 @@ public class TestRangeRetrieverHandler {
Fragment[] frags = sm.splitNG(conf, "employee", meta, tablePath, Integer.MAX_VALUE);
TaskAttemptContext
- ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(),
new Fragment[] {frags[0]}, testDir);
Expr expr = analyzer.parse(SORT_QUERY[1]);
LogicalPlan plan = planner.createPlan(expr);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
index b70dda2..aa2c5fd 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/worker/dataserver/TestHttpDataServer.java
@@ -20,8 +20,10 @@ package org.apache.tajo.worker.dataserver;
import org.apache.hadoop.net.NetUtils;
import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.QueryUnitId;
+import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.util.CommonTestingUtil;
import org.apache.tajo.worker.InterDataRetriever;
import org.apache.tajo.worker.dataserver.retriever.DataRetriever;
@@ -77,8 +79,8 @@ public class TestHttpDataServer {
@Test
public final void testInterDataRetriver() throws Exception {
- ExecutionBlockId schid = QueryIdFactory.newExecutionBlockId(
- QueryIdFactory.newQueryId());
+ MasterPlan plan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
+ ExecutionBlockId schid = plan.newExecutionBlockId();
QueryUnitId qid1 = QueryIdFactory.newQueryUnitId(schid);
QueryUnitId qid2 = QueryIdFactory.newQueryUnitId(schid);
@@ -118,8 +120,8 @@ public class TestHttpDataServer {
@Test(expected = FileNotFoundException.class)
public final void testNoSuchFile() throws Exception {
- ExecutionBlockId schid = QueryIdFactory.newExecutionBlockId(
- QueryIdFactory.newQueryId());
+ MasterPlan plan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
+ ExecutionBlockId schid = plan.newExecutionBlockId();
QueryUnitId qid1 = QueryIdFactory.newQueryUnitId(schid);
QueryUnitId qid2 = QueryIdFactory.newQueryUnitId(schid);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql b/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql
new file mode 100644
index 0000000..66ea3ff
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/queries/complex_union_1.sql
@@ -0,0 +1,29 @@
+create table xdr_url as
+
+SELECT
+ l_orderkey,
+ l_partkey,
+ query
+FROM
+ (
+ SELECT
+ l_orderkey,
+ l_partkey,
+ "abc" as query
+ FROM
+ lineitem
+ WHERE
+ l_orderkey = 1
+
+ UNION ALL
+
+ SELECT
+ l_orderkey,
+ l_partkey,
+ "bbc" as query
+ FROM
+ lineitem
+ WHERE
+ l_orderkey = 1
+) result
+
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/queries/complex_union_2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/queries/complex_union_2.sql b/tajo-core/tajo-core-backend/src/test/queries/complex_union_2.sql
new file mode 100644
index 0000000..d54ed27
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/queries/complex_union_2.sql
@@ -0,0 +1,35 @@
+SELECT *
+FROM
+(
+ SELECT
+ l_orderkey,
+ l_partkey,
+ url
+ FROM
+ (
+ SELECT
+ l_orderkey,
+ l_partkey,
+ CASE
+ WHEN
+ l_partkey IS NOT NULL THEN ''
+ WHEN l_orderkey = 1 THEN '1'
+ ELSE
+ '2'
+ END AS url
+ FROM
+ lineitem
+ ) res1
+ JOIN
+ (
+ SELECT
+ *
+ FROM
+ part
+ ) res2
+ ON l_partkey = p_partkey
+) result
+
+
+
+
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/queries/select_13.hiveql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/queries/select_13.hiveql b/tajo-core/tajo-core-backend/src/test/queries/select_13.hiveql
index c76b1a0..e9f7592 100644
--- a/tajo-core/tajo-core-backend/src/test/queries/select_13.hiveql
+++ b/tajo-core/tajo-core-backend/src/test/queries/select_13.hiveql
@@ -1,2 +1,2 @@
select A.*
-from table1 AS A
\ No newline at end of file
+from table1 A
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/queries/set_4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/queries/set_4.sql b/tajo-core/tajo-core-backend/src/test/queries/set_4.sql
new file mode 100644
index 0000000..a8f0814
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/queries/set_4.sql
@@ -0,0 +1,36 @@
+SELECT *
+FROM
+(
+ SELECT
+ a.reg_date,
+ a.user_id
+ FROM
+ (SELECT buy_date AS bdate
+ FROM buy_history
+ WHERE host='a0') as a JOIN (SELECT * FROM category_info WHERE category_id ='A1') as b ON a.id=b.id
+ UNION ALL
+ SELECT
+ a.reg_date,
+ a.user_id
+ FROM
+ (SELECT buy_date AS bdate
+ FROM buy_history
+ WHERE host='b0') as a JOIN (SELECT * FROM category_info WHERE category_id ='B1') as b ON a.id=b.id
+ UNION ALL
+ SELECT
+ a.reg_date,
+ a.user_id
+ FROM
+ (SELECT buy_date AS bdate
+ FROM buy_history
+ WHERE host='c0') as a JOIN (SELECT * FROM category_info WHERE category_id ='C1') as b ON a.id=b.id
+ UNION ALL
+ SELECT
+ a.reg_date,
+ a.user_id
+ FROM
+ (SELECT buy_date AS bdate
+ FROM buy_history
+ WHERE host='d0') as a JOIN (SELECT * FROM category_info WHERE category_id ='D1') as b ON a.id=b.id
+
+) as T
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/queries/table_subquery2.sql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/queries/table_subquery2.sql b/tajo-core/tajo-core-backend/src/test/queries/table_subquery2.sql
new file mode 100644
index 0000000..206b76f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/queries/table_subquery2.sql
@@ -0,0 +1,21 @@
+SELECT
+ t1.fk,
+ t2.name
+FROM
+ (
+ SELECT
+ table1.fk
+ FROM
+ table1
+ ) t1
+
+ inner join
+
+ (
+ SELECT
+ table2.name
+ FROM
+ table2
+ ) t2
+
+ ON t1.fk = t2.fk;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/resources/catalog-default.xml
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/resources/catalog-default.xml b/tajo-core/tajo-core-backend/src/test/resources/catalog-default.xml
index 3b5316a..05083a4 100644
--- a/tajo-core/tajo-core-backend/src/test/resources/catalog-default.xml
+++ b/tajo-core/tajo-core-backend/src/test/resources/catalog-default.xml
@@ -21,7 +21,7 @@
<configuration>
<property>
- <name>catalog.master.addr</name>
+ <name>tajo.catalog.master.addr</name>
<value>127.0.0.1:9002</value>
</property>
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
index 1e33ad6..7092953 100644
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
+++ b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/TestTupleComparator.java
@@ -66,8 +66,8 @@ public class TestTupleComparator {
DatumFactory.createInt4(4),
DatumFactory.createText("abd")});
- SortSpec sortKey1 = new SortSpec(schema.getColumn("col4"), true, false);
- SortSpec sortKey2 = new SortSpec(schema.getColumn("col5"), true, false);
+ SortSpec sortKey1 = new SortSpec(schema.getColumnByFQN("col4"), true, false);
+ SortSpec sortKey2 = new SortSpec(schema.getColumnByFQN("col5"), true, false);
TupleComparator tc = new TupleComparator(schema,
new SortSpec[] {sortKey1, sortKey2});
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
index 97123c6..166a001 100644
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
+++ b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestBSTIndex.java
@@ -93,8 +93,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", Type.INT8));
@@ -166,8 +166,8 @@ public class TestBSTIndex {
appender.init();
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", Type.INT8));
@@ -256,8 +256,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, status.getLen());
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", Type.INT8));
@@ -325,8 +325,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", Type.INT4));
@@ -413,8 +413,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", Type.INT4));
@@ -497,8 +497,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), false, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), false, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", DataType.LONG));
@@ -570,8 +570,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), false, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), false, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", DataType.LONG));
@@ -638,8 +638,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", DataType.INT));
@@ -720,8 +720,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", DataType.INT));
@@ -793,8 +793,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", DataType.INT));
@@ -876,8 +876,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", DataType.LONG));
@@ -950,8 +950,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", DataType.INT));
@@ -1050,8 +1050,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", DataType.INT));
@@ -1126,8 +1126,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), false, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), false, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), false, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), false, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", DataType.LONG));
@@ -1206,8 +1206,8 @@ public class TestBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen, null);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), false, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), false, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), false, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), false, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", DataType.INT));
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
index 72bdbb0..00a72e4 100644
--- a/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
+++ b/tajo-core/tajo-core-storage/src/test/java/org/apache/tajo/storage/index/TestSingleCSVFileBSTIndex.java
@@ -96,8 +96,8 @@ public class TestSingleCSVFileBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen);
SortSpec[] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("long"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("double"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("long"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("double"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("long", Type.INT8));
@@ -185,8 +185,8 @@ public class TestSingleCSVFileBSTIndex {
Fragment tablet = new Fragment("table1_1", status.getPath(), meta, 0, fileLen);
SortSpec [] sortKeys = new SortSpec[2];
- sortKeys[0] = new SortSpec(schema.getColumn("int"), true, false);
- sortKeys[1] = new SortSpec(schema.getColumn("long"), true, false);
+ sortKeys[0] = new SortSpec(schema.getColumnByFQN("int"), true, false);
+ sortKeys[1] = new SortSpec(schema.getColumnByFQN("long"), true, false);
Schema keySchema = new Schema();
keySchema.addColumn(new Column("int", Type.INT4));
[5/7] TAJO-184: Refactor GlobalPlanner and global plan data
structure. (hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java
index e053c3c..e754a7f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SortNode.java
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
import com.google.gson.annotations.Expose;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.util.TUtil;
public final class SortNode extends UnaryNode implements Cloneable {
@@ -69,7 +70,22 @@ public final class SortNode extends UnaryNode implements Cloneable {
return sort;
}
-
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Sort");
+ StringBuilder sb = new StringBuilder("Sort Keys: ");
+ for (int i = 0; i < sortKeys.length; i++) {
+ sb.append(sortKeys[i].getSortKey().getColumnName()).append(" ")
+ .append(sortKeys[i].isAscending() ? "asc" : "desc");
+ if( i < sortKeys.length - 1) {
+ sb.append(",");
+ }
+ }
+ planStr.addExplan(sb.toString());
+ return planStr;
+ }
+
public String toString() {
StringBuilder sb = new StringBuilder("Sort [key= ");
for (int i = 0; i < sortKeys.length; i++) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
index 817ca35..a2dd097 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/StoreTableNode.java
@@ -22,10 +22,12 @@ import com.google.common.base.Preconditions;
import com.google.gson.annotations.Expose;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.catalog.Options;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.util.TUtil;
import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType.LIST_PARTITION;
public class StoreTableNode extends UnaryNode implements Cloneable {
@Expose private String tableName;
@@ -33,7 +35,6 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
@Expose private PartitionType partitionType;
@Expose private int numPartitions;
@Expose private Column [] partitionKeys;
- @Expose private boolean local;
@Expose private Options options;
@Expose private boolean isCreatedTable = false;
@Expose private boolean isOverwritten = false;
@@ -41,7 +42,6 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
public StoreTableNode(String tableName) {
super(NodeType.STORE);
this.tableName = tableName;
- this.local = false;
}
public final String getTableName() {
@@ -59,14 +59,6 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
public StoreType getStorageType() {
return this.storageType;
}
-
- public final void setLocal(boolean local) {
- this.local = local;
- }
-
- public final boolean isLocal() {
- return this.local;
- }
public final int getNumPartitions() {
return this.numPartitions;
@@ -80,10 +72,10 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
return this.partitionKeys;
}
- public final void setListPartition() {
- this.partitionType = PartitionType.LIST;
+ public final void setDefaultParition() {
+ this.partitionType = LIST_PARTITION;
this.partitionKeys = null;
- this.numPartitions = 0;
+ this.numPartitions = 1;
}
public final void setPartitions(PartitionType type, Column [] keys, int numPartitions) {
@@ -113,6 +105,16 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
return this.options;
}
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Store");
+ planStr.appendTitle(" into ").appendTitle(tableName);
+ planStr.addExplan("Store type: " + storageType);
+
+ return planStr;
+ }
+
public boolean isCreatedTable() {
return isCreatedTable;
}
@@ -152,7 +154,7 @@ public class StoreTableNode extends UnaryNode implements Cloneable {
store.isOverwritten = isOverwritten;
return store;
}
-
+
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("\"Store\": {\"table\": \""+tableName);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
new file mode 100644
index 0000000..8ad8cf5
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/TableSubQueryNode.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.logical;
+
+import com.google.common.base.Objects;
+import com.google.gson.annotations.Expose;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.Target;
+
+public class TableSubQueryNode extends RelationNode implements Projectable {
+ @Expose private String tableName;
+ @Expose private LogicalNode subQuery;
+ @Expose private Target [] targets; // unused
+
+ public TableSubQueryNode(String tableName, LogicalNode subQuery) {
+ super(NodeType.TABLE_SUBQUERY);
+ this.tableName = tableName;
+ this.subQuery = subQuery;
+ setOutSchema(PlannerUtil.getQualifiedSchema(this.subQuery.getOutSchema(), tableName));
+ setInSchema(this.subQuery.getInSchema());
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ @Override
+ public String getCanonicalName() {
+ return tableName;
+ }
+
+ @Override
+ public Schema getTableSchema() {
+ return getOutSchema();
+ }
+
+ public void setSubQuery(LogicalNode node) {
+ this.subQuery = node;
+ setInSchema(subQuery.getInSchema());
+ }
+
+ public LogicalNode getSubQuery() {
+ return subQuery;
+ }
+
+ @Override
+ public boolean hasTargets() {
+ return targets != null;
+ }
+
+ @Override
+ public void setTargets(Target[] targets) {
+ this.targets = targets;
+ }
+
+ @Override
+ public Target[] getTargets() {
+ return targets;
+ }
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("TableSubQuery");
+ planStr.appendTitle(" as ").appendTitle(tableName);
+ return planStr;
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(tableName, subQuery);
+ }
+
+ @Override
+ public boolean equals(Object object) {
+ if (object instanceof TableSubQueryNode) {
+ TableSubQueryNode another = (TableSubQueryNode) object;
+ return tableName.equals(another.tableName) && subQuery.equals(another.subQuery);
+ }
+
+ return false;
+ }
+
+ @Override
+ public Object clone() throws CloneNotSupportedException {
+ TableSubQueryNode newTableSubQueryNode = (TableSubQueryNode) super.clone();
+ newTableSubQueryNode.tableName = tableName;
+ return newTableSubQueryNode;
+ }
+
+ @Override
+ public void preOrder(LogicalNodeVisitor visitor) {
+ visitor.visit(this);
+ subQuery.preOrder(visitor);
+ }
+
+ @Override
+ public void postOrder(LogicalNodeVisitor visitor) {
+ subQuery.preOrder(visitor);
+ visitor.visit(this);
+ }
+
+ public String toString() {
+ return "Table Subquery (alias = " + tableName + ")\n" + subQuery.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java
index 4380996..7f6e065 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnaryNode.java
@@ -16,9 +16,6 @@
* limitations under the License.
*/
-/**
- *
- */
package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
@@ -42,8 +39,8 @@ public abstract class UnaryNode extends LogicalNode implements Cloneable {
this.child = subNode;
}
- public LogicalNode getChild() {
- return this.child;
+ public <T extends LogicalNode> T getChild() {
+ return (T) this.child;
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java
index c0aafbd..a62e91b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/UnionNode.java
@@ -21,6 +21,8 @@
*/
package org.apache.tajo.engine.planner.logical;
+import org.apache.tajo.engine.planner.PlanString;
+
public class UnionNode extends BinaryNode {
public UnionNode() {
@@ -33,6 +35,16 @@ public class UnionNode extends BinaryNode {
setRightChild(inner);
}
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Union");
+ planStr.appendTitle(" (L - " + ((TableSubQueryNode)getLeftChild()).getTableName());
+ planStr.appendTitle(", R - " + ((TableSubQueryNode)getRightChild()).getTableName());
+ planStr.appendTitle(")");
+ return planStr;
+ }
+
public String toString() {
return getLeftChild().toString() + "\n UNION \n" + getRightChild().toString();
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinTree.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinTree.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinTree.java
index d76af7d..2ec055a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinTree.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/join/JoinTree.java
@@ -37,8 +37,8 @@ public class JoinTree {
List<Column> left = EvalTreeUtil.findAllColumnRefs(node.getLeftExpr());
List<Column> right = EvalTreeUtil.findAllColumnRefs(node.getRightExpr());
- String ltbName = left.get(0).getTableName();
- String rtbName = right.get(0).getTableName();
+ String ltbName = left.get(0).getQualifier();
+ String rtbName = right.get(0).getQualifier();
Edge l2r = new Edge(ltbName, rtbName, node);
Edge r2l = new Edge(rtbName, ltbName, node);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java
new file mode 100644
index 0000000..fc569d7
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/BasicPhysicalExecutorVisitor.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import java.util.Stack;
+
+public class BasicPhysicalExecutorVisitor<CONTEXT, RESULT> implements PhysicalExecutorVisitor<CONTEXT, RESULT> {
+
+ @Override
+ public RESULT visitChild(PhysicalExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+
+ if (exec instanceof SeqScanExec) {
+ return visitSeqScan((SeqScanExec) exec, stack, context);
+ } else if (exec instanceof SelectionExec) {
+ return visitSelection((SelectionExec) exec, stack, context);
+ } else if (exec instanceof SortExec) {
+ return visitSort((SortExec) exec, stack, context);
+ } else if (exec instanceof SortAggregateExec) {
+ return visitSortAggregation((SortAggregateExec) exec, stack, context);
+ } else if (exec instanceof ProjectionExec) {
+ return visitProjection((ProjectionExec) exec, stack, context);
+ } else if (exec instanceof HashJoinExec) {
+ return visitHashJoin((HashJoinExec) exec, stack, context);
+ } else if (exec instanceof HashAntiJoinExec) {
+ return visitHashAntiJoin((HashAntiJoinExec) exec, stack, context);
+ } else if (exec instanceof HashSemiJoinExec) {
+ return visitHashSemiJoin((HashSemiJoinExec) exec, stack, context);
+ } else if (exec instanceof LimitExec) {
+ return visitLimit((LimitExec) exec, stack, context);
+ } else {
+ throw new PhysicalPlanningException("Unsupported Type: " + exec.getClass().getSimpleName());
+ }
+ }
+
+ private RESULT visitUnaryExecutor(UnaryPhysicalExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ stack.push(exec);
+ RESULT r = visitChild(exec.getChild(), stack, context);
+ stack.pop();
+ return r;
+ }
+
+ private RESULT visitBinaryExecutor(BinaryPhysicalExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ stack.push(exec);
+ RESULT r = visitChild(exec.getLeftChild(), stack, context);
+ visitChild(exec.getRightChild(), stack, context);
+ stack.pop();
+ return r;
+ }
+
+ @Override
+ public RESULT visitSortAggregation(SortAggregateExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitUnaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitSeqScan(SeqScanExec exec, Stack<PhysicalExec> stack, CONTEXT context) {
+ return null;
+ }
+
+ @Override
+ public RESULT visitSort(SortExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitUnaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitMergeJoin(MergeJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitBinaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitSelection(SelectionExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitUnaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitProjection(ProjectionExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitUnaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitHashJoin(HashJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitBinaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitHashSemiJoin(HashSemiJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitBinaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitHashAntiJoin(HashAntiJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitBinaryExecutor(exec, stack, context);
+ }
+
+ @Override
+ public RESULT visitLimit(LimitExec exec, Stack<PhysicalExec> stack, CONTEXT context)
+ throws PhysicalPlanningException {
+ return visitUnaryExecutor(exec, stack, context);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java
new file mode 100644
index 0000000..131fbe5
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalExecutorVisitor.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import org.apache.tajo.engine.planner.physical.*;
+
+import java.util.Stack;
+
+public interface PhysicalExecutorVisitor<CONTEXT, RESULT> {
+ RESULT visitChild(PhysicalExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitSeqScan(SeqScanExec exec, Stack<PhysicalExec> stack, CONTEXT context);
+ RESULT visitSort(SortExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitSortAggregation(SortAggregateExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitMergeJoin(MergeJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitSelection(SelectionExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitProjection(ProjectionExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitHashJoin(HashJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitHashSemiJoin(HashSemiJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitHashAntiJoin(HashAntiJoinExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+ RESULT visitLimit(LimitExec exec, Stack<PhysicalExec> stack, CONTEXT context) throws PhysicalPlanningException;
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
new file mode 100644
index 0000000..bd773ed
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanUtil.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import java.util.Stack;
+
+public class PhysicalPlanUtil {
+ public static <T extends PhysicalExec> T findExecutor(PhysicalExec plan, Class<? extends PhysicalExec> clazz)
+ throws PhysicalPlanningException {
+ return (T) new FindVisitor().visitChild(plan, new Stack<PhysicalExec>(), clazz);
+ }
+
+ private static class FindVisitor extends BasicPhysicalExecutorVisitor<Class<? extends PhysicalExec>, PhysicalExec> {
+ public PhysicalExec visitChild(PhysicalExec exec, Stack<PhysicalExec> stack, Class<? extends PhysicalExec> target)
+ throws PhysicalPlanningException {
+
+ if (target.isAssignableFrom(exec.getClass())) {
+ return exec;
+ } else {
+ return super.visitChild(exec, stack, target);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java
new file mode 100644
index 0000000..0d7554d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/PhysicalPlanningException.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.physical;
+
+import java.io.IOException;
+
+public class PhysicalPlanningException extends IOException {
+ public PhysicalPlanningException(String message) {
+ super(message);
+ }
+
+ public PhysicalPlanningException(IOException ioe) {
+ super(ioe);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
index 9051219..2d736ce 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java
@@ -136,6 +136,6 @@ public class SeqScanExec extends PhysicalExec {
}
public String getTableName() {
- return plan.getTableId();
+ return plan.getTableName();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java
index 1f771e6..a33cbd7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/FilterPushDownRule.java
@@ -43,8 +43,7 @@ public class FilterPushDownRule extends BasicLogicalPlanVisitor<List<EvalNode>>
@Override
public boolean isEligible(LogicalPlan plan) {
for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
- LogicalNode toBeOptimized = block.getRoot();
- if (PlannerUtil.findTopNode(toBeOptimized, NodeType.SELECTION) != null) {
+ if (PlannerUtil.findTopNode(block.getRoot(), NodeType.SELECTION) != null) {
return true;
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java
index 4c2f9d7..1484f32 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/rewrite/ProjectionPushDownRule.java
@@ -49,7 +49,7 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
public boolean isEligible(LogicalPlan plan) {
LogicalNode toBeOptimized = plan.getRootBlock().getRoot();
- if (PlannerUtil.checkIfDDLPlan(toBeOptimized) && !plan.getRootBlock().hasTableExpression()) {
+ if (PlannerUtil.checkIfDDLPlan(toBeOptimized) || !plan.getRootBlock().hasTableExpression()) {
LOG.info("This query skips the logical optimization step.");
return false;
}
@@ -59,25 +59,29 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
@Override
public LogicalPlan rewrite(LogicalPlan plan) throws PlanningException {
- for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
- NodeType nodeType = block.getRootType();
- // skip a non-table-expression block.
- if (!(nodeType == NodeType.INSERT || nodeType == NodeType.CREATE_TABLE || nodeType == NodeType.EXPRS)) {
- Stack<LogicalNode> stack = new Stack<LogicalNode>();
- PushDownContext context = new PushDownContext(block);
- context.plan = plan;
- if (block.getProjection() != null &&
- block.getProjection().isAllProjected()) {
- context.targetListManager = new TargetListManager(plan,
- block.getProjectionNode().getTargets());
- } else {
- context.targetListManager= new TargetListManager(plan, block.getName());
- }
- context.upperRequired = new HashSet<Column>(block.getSchema().getColumns());
- visitChild(plan, block.getRoot(), stack, context);
- }
+ LogicalPlan.QueryBlock rootBlock = plan.getRootBlock();
+
+ LogicalPlan.QueryBlock topmostBlock;
+
+ // skip a non-table-expression block.
+ if (plan.getRootBlock().getRootType() == NodeType.INSERT) {
+ topmostBlock = plan.getChildBlocks(rootBlock).iterator().next();
+ } else {
+ topmostBlock = rootBlock;
}
+ Stack<LogicalNode> stack = new Stack<LogicalNode>();
+ PushDownContext context = new PushDownContext(topmostBlock);
+ context.plan = plan;
+
+ if (topmostBlock.getProjection() != null && topmostBlock.getProjection().isAllProjected()) {
+ context.targetListManager = new TargetListManager(plan, topmostBlock.getProjectionNode().getTargets());
+ } else {
+ context.targetListManager= new TargetListManager(plan, topmostBlock.getName());
+ }
+
+ visitChild(plan, topmostBlock.getRoot(), stack, context);
+
return plan;
}
@@ -113,8 +117,24 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
@Override
public LogicalNode visitProjection(LogicalPlan plan, ProjectionNode node, Stack<LogicalNode> stack,
PushDownContext context) throws PlanningException {
- for (Target target : node.getTargets()) {
- context.upperRequired.add(target.getColumnSchema());
+ if (context.upperRequired == null) { // all projected
+ context.upperRequired = new HashSet<Column>();
+ for (Target target : node.getTargets()) {
+ context.upperRequired.add(target.getColumnSchema());
+ }
+ } else {
+ List<Target> projectedTarget = new ArrayList<Target>();
+ for (Target target : node.getTargets()) {
+ if (context.upperRequired.contains(target.getColumnSchema())) {
+ projectedTarget.add(target);
+ }
+ }
+ node.setTargets(projectedTarget.toArray(new Target[projectedTarget.size()]));
+
+ context.upperRequired = new HashSet<Column>();
+ for (Target target : node.getTargets()) {
+ context.upperRequired.add(target.getColumnSchema());
+ }
}
stack.push(node);
@@ -126,13 +146,14 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
// If all expressions are evaluated in the child operators and the last operator is projectable,
// ProjectionNode will not be necessary. It eliminates ProjectionNode.
if (context.targetListManager.isAllEvaluated() && (childNode instanceof Projectable)) {
+ child.setOutSchema(context.targetListManager.getUpdatedSchema());
if (stack.isEmpty()) {
// update the child node's output schemas
- child.setOutSchema(context.targetListManager.getUpdatedSchema());
context.queryBlock.setRoot(child);
+ } else if (stack.peek().getType() == NodeType.TABLE_SUBQUERY) {
+ ((TableSubQueryNode)stack.peek()).setSubQuery(childNode);
} else {
LogicalNode parent = stack.peek();
- child.setOutSchema(context.targetListManager.getUpdatedSchema());
PlannerUtil.deleteNode(parent, node);
}
return child;
@@ -242,6 +263,42 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
}
@Override
+ public LogicalNode visitTableSubQuery(LogicalPlan plan, TableSubQueryNode node, Stack<LogicalNode> stack,
+ PushDownContext context) throws PlanningException {
+ LogicalPlan.QueryBlock subBlock = plan.getBlock(node.getSubQuery());
+ LogicalNode subRoot = subBlock.getRoot();
+
+ Stack<LogicalNode> newStack = new Stack<LogicalNode>();
+ newStack.push(node);
+ PushDownContext newContext = new PushDownContext(subBlock);
+ if (subBlock.getProjection() != null && subBlock.getProjection().isAllProjected()
+ && context.upperRequired.size() == 0) {
+ newContext.targetListManager = new TargetListManager(plan, subBlock.getProjectionNode().getTargets());
+ } else {
+ List<Target> projectedTarget = new ArrayList<Target>();
+ for (Target target : subBlock.getTargetListManager().getUnEvaluatedTargets()) {
+ for (Column column : context.upperRequired) {
+ if (column.hasQualifier() && !node.getTableName().equals(column.getQualifier())) {
+ continue;
+ }
+ if (target.getColumnSchema().getColumnName().equalsIgnoreCase(column.getColumnName())) {
+ projectedTarget.add(target);
+ }
+ }
+ }
+ newContext.targetListManager = new TargetListManager(plan, projectedTarget.toArray(new Target[projectedTarget.size()]));
+ }
+
+ newContext.upperRequired = new HashSet<Column>();
+ newContext.upperRequired.addAll(PlannerUtil.targetToSchema(newContext.targetListManager.getTargets()).getColumns());
+
+ LogicalNode child = visitChild(plan, subRoot, newStack, newContext);
+ newStack.pop();
+ node.setInSchema(PlannerUtil.getQualifiedSchema(child.getOutSchema(), node.getTableName()));
+ return pushDownProjectablePost(context, node, isTopmostProjectable(stack));
+ }
+
+ @Override
public LogicalNode visitScan(LogicalPlan plan, ScanNode node, Stack<LogicalNode> stack, PushDownContext context)
throws PlanningException {
return pushDownProjectablePost(context, node, isTopmostProjectable(stack));
@@ -280,7 +337,7 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
if (!targetListManager.isEvaluated(i) && PlannerUtil.canBeEvaluated(expr, node)) {
- if (node instanceof ScanNode) { // For ScanNode
+ if (node instanceof RelationNode) { // For ScanNode
if (expr.getType() == EvalType.FIELD && !targetListManager.getTarget(i).hasAlias()) {
targetListManager.setEvaluated(i);
@@ -306,7 +363,7 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
Projectable projectable = (Projectable) node;
if (last) {
- Preconditions.checkState(targetListManager.isAllEvaluated(), "Not all targets are evaluated.");
+ Preconditions.checkState(targetListManager.isAllEvaluated(), "Not all targets are evaluated");
projectable.setTargets(targetListManager.getTargets());
targetListManager.getUpdatedTarget();
node.setOutSchema(targetListManager.getUpdatedSchema());
@@ -361,18 +418,38 @@ public class ProjectionPushDownRule extends BasicLogicalPlanVisitor<ProjectionPu
return true;
}
+ private TargetListManager buildSubBlockTargetList(LogicalPlan plan,
+ LogicalPlan.QueryBlock subQueryBlock, TableSubQueryNode subQueryNode, Set<Column> upperRequired) {
+ TargetListManager subBlockTargetList;
+ List<Target> projectedTarget = new ArrayList<Target>();
+ for (Target target : subQueryBlock.getTargetListManager().getUnEvaluatedTargets()) {
+ for (Column column : upperRequired) {
+ if (!subQueryNode.getTableName().equals(column.getQualifier())) {
+ continue;
+ }
+ if (target.getColumnSchema().getColumnName().equalsIgnoreCase(column.getColumnName())) {
+ projectedTarget.add(target);
+ }
+ }
+ }
+ subBlockTargetList = new TargetListManager(plan, projectedTarget.toArray(new Target[projectedTarget.size()]));
+ return subBlockTargetList;
+ }
+
private BinaryNode pushDownSetNode(LogicalPlan plan, BinaryNode setNode, Stack<LogicalNode> stack,
PushDownContext context) throws PlanningException {
- LogicalPlan.QueryBlock leftBlock = plan.getBlock(setNode.getLeftChild());
+ LogicalPlan.QueryBlock currentBlock = plan.getBlock(setNode);
+ LogicalPlan.QueryBlock leftBlock = plan.getChildBlocks(currentBlock).get(0);
+ LogicalPlan.QueryBlock rightBlock = plan.getChildBlocks(currentBlock).get(1);
+
PushDownContext leftContext = new PushDownContext(context, leftBlock);
- leftContext.targetListManager = new TargetListManager(plan,
- leftBlock.getTargetListManager().getUnEvaluatedTargets());
+ leftContext.targetListManager = buildSubBlockTargetList(plan, leftBlock,
+ (TableSubQueryNode) setNode.getLeftChild(), context.upperRequired);
- LogicalPlan.QueryBlock rightBlock = plan.getBlock(setNode.getRightChild());
PushDownContext rightContext = new PushDownContext(context, rightBlock);
- rightContext.targetListManager = new TargetListManager(plan,
- rightBlock.getTargetListManager().getUnEvaluatedTargets());
+ rightContext.targetListManager = buildSubBlockTargetList(plan, rightBlock,
+ (TableSubQueryNode) setNode.getRightChild(), context.upperRequired);
stack.push(setNode);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
index a0422af..0676277 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/QueryUnitRequestImpl.java
@@ -18,6 +18,7 @@
package org.apache.tajo.engine.query;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.ipc.TajoWorkerProtocol.Fetch;
import org.apache.tajo.ipc.TajoWorkerProtocol.QueryUnitRequestProto;
@@ -42,6 +43,7 @@ public class QueryUnitRequestImpl implements QueryUnitRequest {
private List<Fetch> fetches;
private Boolean shouldDie;
private QueryContext queryContext;
+ private DataChannel dataChannel;
private QueryUnitRequestProto proto = QueryUnitRequestProto.getDefaultInstance();
private QueryUnitRequestProto.Builder builder = null;
@@ -55,9 +57,9 @@ public class QueryUnitRequestImpl implements QueryUnitRequest {
public QueryUnitRequestImpl(QueryUnitAttemptId id, List<Fragment> fragments,
String outputTable, boolean clusteredOutput,
- String serializedData, QueryContext queryContext) {
+ String serializedData, QueryContext queryContext, DataChannel channel) {
this();
- this.set(id, fragments, outputTable, clusteredOutput, serializedData, queryContext);
+ this.set(id, fragments, outputTable, clusteredOutput, serializedData, queryContext, channel);
}
public QueryUnitRequestImpl(QueryUnitRequestProto proto) {
@@ -68,8 +70,8 @@ public class QueryUnitRequestImpl implements QueryUnitRequest {
}
public void set(QueryUnitAttemptId id, List<Fragment> fragments,
- String outputTable, boolean clusteredOutput,
- String serializedData, QueryContext queryContext) {
+ String outputTable, boolean clusteredOutput,
+ String serializedData, QueryContext queryContext, DataChannel dataChannel) {
this.id = id;
this.fragments = fragments;
this.outputTable = outputTable;
@@ -77,6 +79,8 @@ public class QueryUnitRequestImpl implements QueryUnitRequest {
this.serializedData = serializedData;
this.isUpdated = true;
this.queryContext = queryContext;
+ this.queryContext = queryContext;
+ this.dataChannel = dataChannel;
}
@Override
@@ -198,6 +202,24 @@ public class QueryUnitRequestImpl implements QueryUnitRequest {
maybeInitBuilder();
this.queryContext = queryContext;
}
+
+ public void setDataChannel(DataChannel dataChannel) {
+ maybeInitBuilder();
+ this.dataChannel = dataChannel;
+ }
+
+ @Override
+ public DataChannel getDataChannel() {
+ QueryUnitRequestProtoOrBuilder p = viaProto ? proto : builder;
+ if (dataChannel != null) {
+ return dataChannel;
+ }
+ if (!p.hasQueryContext()) {
+ return null;
+ }
+ this.dataChannel = new DataChannel(p.getDataChannel());
+ return this.dataChannel;
+ }
public List<Fetch> getFetches() {
initFetches();
@@ -272,6 +294,9 @@ public class QueryUnitRequestImpl implements QueryUnitRequest {
if (this.queryContext != null) {
builder.setQueryContext(queryContext.getProto());
}
+ if (this.dataChannel != null) {
+ builder.setDataChannel(dataChannel.getProto());
+ }
}
private void mergeLocalToProto() {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetMetaDataImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetMetaDataImpl.java
index 9ebd158..7a9c4e0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetMetaDataImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/query/ResultSetMetaDataImpl.java
@@ -179,7 +179,7 @@ public class ResultSetMetaDataImpl implements ResultSetMetaData {
*/
@Override
public String getTableName(int column) throws SQLException {
- return meta.getSchema().getColumn(column - 1).getTableName();
+ return meta.getSchema().getColumn(column - 1).getQualifier();
}
/* (non-Javadoc)
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
index 0775c3c..a9f3706 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/ipc/protocolrecords/QueryUnitRequest.java
@@ -21,6 +21,7 @@
*/
package org.apache.tajo.ipc.protocolrecords;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.common.ProtoObject;
import org.apache.tajo.ipc.TajoWorkerProtocol;
@@ -44,4 +45,5 @@ public interface QueryUnitRequest extends ProtoObject<TajoWorkerProtocol.QueryUn
public boolean shouldDie();
public void setShouldDie();
public QueryContext getQueryContext();
+ public DataChannel getDataChannel();
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
index a92ef75..d509156 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlock.java
@@ -17,10 +17,13 @@ package org.apache.tajo.master;
import com.google.common.base.Preconditions;
import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.engine.planner.PlannerUtil;
import org.apache.tajo.engine.planner.logical.*;
import java.util.*;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType;
+
/**
* A distributed execution plan (DEP) is a direct acyclic graph (DAG) of ExecutionBlocks.
* An ExecutionBlock is a basic execution unit that could be distributed across a number of nodes.
@@ -29,17 +32,6 @@ import java.util.*;
* In addition, it includes a logical plan to be executed in each node.
*/
public class ExecutionBlock {
-
- public static enum PartitionType {
- /** for hash partitioning */
- HASH,
- LIST,
- /** for map-side join */
- BROADCAST,
- /** for range partitioning */
- RANGE
- }
-
private ExecutionBlockId executionBlockId;
private LogicalNode plan = null;
private StoreTableNode store = null;
@@ -47,9 +39,12 @@ public class ExecutionBlock {
private ExecutionBlock parent;
private Map<ScanNode, ExecutionBlock> childSubQueries = new HashMap<ScanNode, ExecutionBlock>();
private PartitionType outputType;
+
private boolean hasJoinPlan;
private boolean hasUnionPlan;
+ private Set<String> broadcasted = new HashSet<String>();
+
public ExecutionBlock(ExecutionBlockId executionBlockId) {
this.executionBlockId = executionBlockId;
}
@@ -58,10 +53,6 @@ public class ExecutionBlock {
return executionBlockId;
}
- public String getOutputName() {
- return store.getTableName();
- }
-
public void setPartitionType(PartitionType partitionType) {
this.outputType = partitionType;
}
@@ -72,10 +63,9 @@ public class ExecutionBlock {
public void setPlan(LogicalNode plan) {
hasJoinPlan = false;
- Preconditions.checkArgument(plan.getType() == NodeType.STORE);
-
+ hasUnionPlan = false;
+ this.scanlist.clear();
this.plan = plan;
- store = (StoreTableNode) plan;
LogicalNode node = plan;
ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
@@ -96,6 +86,9 @@ public class ExecutionBlock {
s.add(s.size(), binary.getRightChild());
} else if (node instanceof ScanNode) {
scanlist.add((ScanNode)node);
+ } else if (node instanceof TableSubQueryNode) {
+ TableSubQueryNode subQuery = (TableSubQueryNode) node;
+ s.add(s.size(), subQuery.getSubQuery());
}
}
}
@@ -105,6 +98,10 @@ public class ExecutionBlock {
return plan;
}
+ public boolean isRoot() {
+ return !hasParentBlock() || !(getParentBlock().hasParentBlock()) && getParentBlock().hasUnion();
+ }
+
public boolean hasParentBlock() {
return parent != null;
}
@@ -173,4 +170,24 @@ public class ExecutionBlock {
public boolean hasUnion() {
return hasUnionPlan;
}
+
+ public void addBroadcastTables(Collection<String> tableNames) {
+ broadcasted.addAll(tableNames);
+ }
+
+ public void addBroadcastTable(String tableName) {
+ broadcasted.add(tableName);
+ }
+
+ public boolean isBroadcastTable(String tableName) {
+ return broadcasted.contains(tableName);
+ }
+
+ public Collection<String> getBroadcastTables() {
+ return broadcasted;
+ }
+
+ public String toString() {
+ return executionBlockId.toString();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlockCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlockCursor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlockCursor.java
index fd3ae1e..51c825c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlockCursor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/ExecutionBlockCursor.java
@@ -25,10 +25,12 @@ import java.util.Iterator;
* For each call of nextBlock(), it retrieves a next ExecutionBlock in a postfix order.
*/
public class ExecutionBlockCursor {
+ private MasterPlan masterPlan;
private ArrayList<ExecutionBlock> orderedBlocks = new ArrayList<ExecutionBlock>();
private int cursor = 0;
public ExecutionBlockCursor(MasterPlan plan) {
+ this.masterPlan = plan;
buildOrder(plan.getRoot());
}
@@ -37,26 +39,14 @@ public class ExecutionBlockCursor {
}
private void buildOrder(ExecutionBlock current) {
- if (current.hasChildBlock()) {
- if (current.getChildNum() == 1) {
- ExecutionBlock block = current.getChildBlocks().iterator().next();
+ if (!masterPlan.isLeaf(current.getId())) {
+ if (masterPlan.getChildCount(current.getId()) == 1) {
+ ExecutionBlock block = masterPlan.getChild(current, 0);
buildOrder(block);
} else {
- Iterator<ExecutionBlock> it = current.getChildBlocks().iterator();
- ExecutionBlock outer = it.next();
- ExecutionBlock inner = it.next();
-
- // Switch between outer and inner
- // if an inner has a child and an outer doesn't.
- // It is for left-deep-first search.
- if (!outer.hasChildBlock() && inner.hasChildBlock()) {
- ExecutionBlock tmp = outer;
- outer = inner;
- inner = tmp;
+ for (ExecutionBlock exec : masterPlan.getChilds(current)) {
+ buildOrder(exec);
}
-
- buildOrder(outer);
- buildOrder(inner);
}
}
orderedBlocks.add(current);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
index 2ddd891..5080599 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java
@@ -55,7 +55,6 @@ import org.apache.tajo.storage.StorageUtil;
import java.io.IOException;
import java.sql.SQLException;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.List;
import static org.apache.tajo.ipc.ClientProtos.GetQueryStatusResponse;
@@ -107,7 +106,8 @@ public class GlobalEngine extends AbstractService {
NoSuchQueryIdException, IllegalQueryStatusException,
UnknownWorkerException, EmptyClusterException {
- LOG.info(">>>>>SQL: " + sql);
+ LOG.info("SQL: " + sql);
+ QueryContext queryContext = new QueryContext();
try {
// setting environment variables
@@ -127,6 +127,10 @@ public class GlobalEngine extends AbstractService {
final boolean hiveQueryMode = context.getConf().getBoolVar(TajoConf.ConfVars.HIVE_QUERY_MODE);
LOG.info("hive.query.mode:" + hiveQueryMode);
+ if (hiveQueryMode) {
+ queryContext.setHiveQueryMode();
+ }
+
Expr planningContext = hiveQueryMode ? converter.parse(sql) : analyzer.parse(sql);
LogicalPlan plan = createLogicalPlan(planningContext);
@@ -139,7 +143,6 @@ public class GlobalEngine extends AbstractService {
responseBuilder.setResultCode(ClientProtos.ResultCode.OK);
responseBuilder.setState(TajoProtos.QueryState.QUERY_SUCCEEDED);
} else {
- QueryContext queryContext = new QueryContext();
hookManager.doHooks(queryContext, plan);
QueryJobManager queryJobManager = this.context.getQueryJobManager();
@@ -300,7 +303,7 @@ public class GlobalEngine extends AbstractService {
void hook(QueryContext queryContext, LogicalPlan plan) throws Exception;
}
- public class DistributedQueryHookManager {
+ public static class DistributedQueryHookManager {
private List<DistributedQueryHook> hooks = new ArrayList<DistributedQueryHook>();
public void addHook(DistributedQueryHook hook) {
hooks.add(hook);
@@ -319,7 +322,7 @@ public class GlobalEngine extends AbstractService {
}
}
- private class CreateTableHook implements DistributedQueryHook {
+ public class CreateTableHook implements DistributedQueryHook {
@Override
public boolean isEligible(QueryContext queryContext, LogicalPlan plan) {
@@ -341,7 +344,7 @@ public class GlobalEngine extends AbstractService {
}
}
- private class InsertHook implements DistributedQueryHook {
+ public static class InsertHook implements DistributedQueryHook {
@Override
public boolean isEligible(QueryContext queryContext, LogicalPlan plan) {
@@ -416,10 +419,8 @@ public class GlobalEngine extends AbstractService {
ProjectionNode projectionNode = new ProjectionNode(targets);
projectionNode.setInSchema(insertNode.getSubQuery().getOutSchema());
projectionNode.setOutSchema(PlannerUtil.targetToSchema(targets));
- Collection<QueryBlockGraph.BlockEdge> edges = plan.getConnectedBlocks(LogicalPlan.ROOT_BLOCK);
- LogicalPlan.QueryBlock block = plan.getBlock(edges.iterator().next().getTargetBlock());
- projectionNode.setChild(block.getRoot());
-
+ List<LogicalPlan.QueryBlock> blocks = plan.getChildBlocks(plan.getRootBlock());
+ projectionNode.setChild(blocks.get(0).getRoot());
storeNode.setOutSchema(projectionNode.getOutSchema());
storeNode.setInSchema(projectionNode.getOutSchema());
@@ -427,12 +428,10 @@ public class GlobalEngine extends AbstractService {
} else {
storeNode.setOutSchema(subQueryOutSchema);
storeNode.setInSchema(subQueryOutSchema);
- Collection<QueryBlockGraph.BlockEdge> edges = plan.getConnectedBlocks(LogicalPlan.ROOT_BLOCK);
- LogicalPlan.QueryBlock block = plan.getBlock(edges.iterator().next().getTargetBlock());
- storeNode.setChild(block.getRoot());
+ List<LogicalPlan.QueryBlock> childBlocks = plan.getChildBlocks(plan.getRootBlock());
+ storeNode.setChild(childBlocks.get(0).getRoot());
}
- storeNode.setListPartition();
if (insertNode.hasStorageType()) {
storeNode.setStorageType(insertNode.getStorageType());
}
[2/7] TAJO-184: Refactor GlobalPlanner and global plan data
structure. (hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
index 7f95a85..b8e2fc3 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlanner.java
@@ -145,14 +145,14 @@ public class TestLogicalPlanner {
testJsonSerDerObject(root);
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projNode = (ProjectionNode) root.getChild();
+ ProjectionNode projNode = root.getChild();
assertEquals(NodeType.SELECTION, projNode.getChild().getType());
- SelectionNode selNode = (SelectionNode) projNode.getChild();
+ SelectionNode selNode = projNode.getChild();
assertEquals(NodeType.SCAN, selNode.getChild().getType());
- ScanNode scanNode = (ScanNode) selNode.getChild();
- assertEquals("employee", scanNode.getTableId());
+ ScanNode scanNode = selNode.getChild();
+ assertEquals("employee", scanNode.getTableName());
}
public static void assertSchema(Schema expected, Schema schema) {
@@ -190,22 +190,17 @@ public class TestLogicalPlanner {
}
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projNode = (ProjectionNode) root.getChild();
+ ProjectionNode projNode = root.getChild();
assertEquals(NodeType.JOIN, projNode.getChild().getType());
- JoinNode joinNode = (JoinNode) projNode.getChild();
+ JoinNode joinNode = projNode.getChild();
assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
- ScanNode leftNode = (ScanNode) joinNode.getLeftChild();
- assertEquals("employee", leftNode.getTableId());
+ ScanNode leftNode = joinNode.getLeftChild();
+ assertEquals("employee", leftNode.getTableName());
assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
- ScanNode rightNode = (ScanNode) joinNode.getRightChild();
- assertEquals("dept", rightNode.getTableId());
- /*
- LogicalNode optimized = LogicalOptimizer.optimize(expr, plan);
- assertSchema(expectedSchema, optimized.getOutSchema());
- */
-
+ ScanNode rightNode = joinNode.getRightChild();
+ assertEquals("dept", rightNode.getTableName());
// three relations
expr = sqlAnalyzer.parse(QUERIES[2]);
@@ -220,30 +215,27 @@ public class TestLogicalPlanner {
root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- projNode = (ProjectionNode) root.getChild();
+ projNode = root.getChild();
assertEquals(NodeType.JOIN, projNode.getChild().getType());
- joinNode = (JoinNode) projNode.getChild();
+ joinNode = projNode.getChild();
assertEquals(NodeType.JOIN, joinNode.getLeftChild().getType());
assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
- ScanNode scan1 = (ScanNode) joinNode.getRightChild();
- assertEquals("score", scan1.getTableId());
+ ScanNode scan1 = joinNode.getRightChild();
+ assertEquals("score", scan1.getTableName());
- JoinNode leftNode2 = (JoinNode) joinNode.getLeftChild();
+ JoinNode leftNode2 = joinNode.getLeftChild();
assertEquals(NodeType.JOIN, leftNode2.getType());
assertEquals(NodeType.SCAN, leftNode2.getLeftChild().getType());
- ScanNode leftScan = (ScanNode) leftNode2.getLeftChild();
- assertEquals("employee", leftScan.getTableId());
+ ScanNode leftScan = leftNode2.getLeftChild();
+ assertEquals("employee", leftScan.getTableName());
assertEquals(NodeType.SCAN, leftNode2.getRightChild().getType());
- ScanNode rightScan = (ScanNode) leftNode2.getRightChild();
- assertEquals("dept", rightScan.getTableId());
- /*
- optimized = LogicalOptimizer.optimize(expr, plan);
- assertSchema(expectedSchema, optimized.getOutSchema());*/
+ ScanNode rightScan = leftNode2.getRightChild();
+ assertEquals("dept", rightScan.getTableName());
}
@@ -273,29 +265,24 @@ public class TestLogicalPlanner {
assertEquals(NodeType.ROOT, plan.getType());
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode proj = (ProjectionNode) root.getChild();
+ ProjectionNode proj = root.getChild();
assertEquals(NodeType.JOIN, proj.getChild().getType());
- JoinNode join = (JoinNode) proj.getChild();
+ JoinNode join = proj.getChild();
assertEquals(JoinType.INNER, join.getJoinType());
assertEquals(NodeType.SCAN, join.getRightChild().getType());
assertTrue(join.hasJoinQual());
- ScanNode scan = (ScanNode) join.getRightChild();
- assertEquals("score", scan.getTableId());
+ ScanNode scan = join.getRightChild();
+ assertEquals("score", scan.getTableName());
assertEquals(NodeType.JOIN, join.getLeftChild().getType());
- join = (JoinNode) join.getLeftChild();
+ join = join.getLeftChild();
assertEquals(JoinType.INNER, join.getJoinType());
assertEquals(NodeType.SCAN, join.getLeftChild().getType());
- ScanNode outer = (ScanNode) join.getLeftChild();
- assertEquals("employee", outer.getTableId());
+ ScanNode outer = join.getLeftChild();
+ assertEquals("employee", outer.getTableName());
assertEquals(NodeType.SCAN, join.getRightChild().getType());
- ScanNode inner = (ScanNode) join.getRightChild();
- assertEquals("dept", inner.getTableId());
-
- /*
- LogicalNode optimized = LogicalOptimizer.optimize(context, plan);
- assertSchema(expectedJoinSchema, optimized.getOutSchema());
- */
+ ScanNode inner = join.getRightChild();
+ assertEquals("dept", inner.getTableName());
}
@Test
@@ -309,29 +296,25 @@ public class TestLogicalPlanner {
assertEquals(NodeType.ROOT, plan.getType());
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode proj = (ProjectionNode) root.getChild();
+ ProjectionNode proj = root.getChild();
assertEquals(NodeType.JOIN, proj.getChild().getType());
- JoinNode join = (JoinNode) proj.getChild();
+ JoinNode join = proj.getChild();
assertEquals(JoinType.INNER, join.getJoinType());
assertEquals(NodeType.SCAN, join.getRightChild().getType());
- ScanNode scan = (ScanNode) join.getRightChild();
- assertEquals("score", scan.getTableId());
+ ScanNode scan = join.getRightChild();
+ assertEquals("score", scan.getTableName());
assertEquals(NodeType.JOIN, join.getLeftChild().getType());
- join = (JoinNode) join.getLeftChild();
+ join = join.getLeftChild();
assertEquals(JoinType.INNER, join.getJoinType());
assertEquals(NodeType.SCAN, join.getLeftChild().getType());
- ScanNode outer = (ScanNode) join.getLeftChild();
- assertEquals("employee", outer.getTableId());
+ ScanNode outer = join.getLeftChild();
+ assertEquals("employee", outer.getTableName());
assertEquals(NodeType.SCAN, join.getRightChild().getType());
- ScanNode inner = (ScanNode) join.getRightChild();
- assertEquals("dept", inner.getTableId());
+ ScanNode inner = join.getRightChild();
+ assertEquals("dept", inner.getTableName());
assertTrue(join.hasJoinQual());
assertEquals(EvalType.EQUAL, join.getJoinQual().getType());
- /*
- LogicalNode optimized = LogicalOptimizer.optimize(expr, plan);
- assertSchema(expectedJoinSchema, optimized.getOutSchema());
- */
}
@Test
@@ -345,30 +328,25 @@ public class TestLogicalPlanner {
assertEquals(NodeType.ROOT, plan.getType());
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode proj = (ProjectionNode) root.getChild();
+ ProjectionNode proj = root.getChild();
assertEquals(NodeType.JOIN, proj.getChild().getType());
- JoinNode join = (JoinNode) proj.getChild();
+ JoinNode join = proj.getChild();
assertEquals(JoinType.RIGHT_OUTER, join.getJoinType());
assertEquals(NodeType.SCAN, join.getRightChild().getType());
- ScanNode scan = (ScanNode) join.getRightChild();
- assertEquals("score", scan.getTableId());
+ ScanNode scan = join.getRightChild();
+ assertEquals("score", scan.getTableName());
assertEquals(NodeType.JOIN, join.getLeftChild().getType());
- join = (JoinNode) join.getLeftChild();
+ join = join.getLeftChild();
assertEquals(JoinType.LEFT_OUTER, join.getJoinType());
assertEquals(NodeType.SCAN, join.getLeftChild().getType());
- ScanNode outer = (ScanNode) join.getLeftChild();
- assertEquals("employee", outer.getTableId());
+ ScanNode outer = join.getLeftChild();
+ assertEquals("employee", outer.getTableName());
assertEquals(NodeType.SCAN, join.getRightChild().getType());
- ScanNode inner = (ScanNode) join.getRightChild();
- assertEquals("dept", inner.getTableId());
+ ScanNode inner = join.getRightChild();
+ assertEquals("dept", inner.getTableName());
assertTrue(join.hasJoinQual());
assertEquals(EvalType.EQUAL, join.getJoinQual().getType());
-
- /*
- LogicalNode optimized = LogicalOptimizer.optimize(context, plan);
- assertSchema(expectedJoinSchema, optimized.getOutSchema());
- */
}
@@ -392,19 +370,19 @@ public class TestLogicalPlanner {
root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projNode = (ProjectionNode) root.getChild();
+ ProjectionNode projNode = root.getChild();
assertEquals(NodeType.GROUP_BY, projNode.getChild().getType());
- GroupbyNode groupByNode = (GroupbyNode) projNode.getChild();
+ GroupbyNode groupByNode = projNode.getChild();
assertEquals(NodeType.JOIN, groupByNode.getChild().getType());
- JoinNode joinNode = (JoinNode) groupByNode.getChild();
+ JoinNode joinNode = groupByNode.getChild();
assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
- ScanNode leftNode = (ScanNode) joinNode.getLeftChild();
- assertEquals("dept", leftNode.getTableId());
+ ScanNode leftNode = joinNode.getLeftChild();
+ assertEquals("dept", leftNode.getTableName());
assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
- ScanNode rightNode = (ScanNode) joinNode.getRightChild();
- assertEquals("score", rightNode.getTableId());
+ ScanNode rightNode = joinNode.getRightChild();
+ assertEquals("score", rightNode.getTableName());
//LogicalOptimizer.optimize(context, plan);
}
@@ -418,9 +396,6 @@ public class TestLogicalPlanner {
testJsonSerDerObject(plan);
Schema expected = tpch.getOutSchema("q2");
assertSchema(expected, plan.getOutSchema());
-// LogicalNode optimized = LogicalOptimizer.optimize(context, plan);
-// System.out.println(optimized);
-// assertSchema(expected, optimized.getOutSchema());
}
@@ -428,17 +403,17 @@ public class TestLogicalPlanner {
assertEquals(NodeType.PROJECTION, plan.getType());
ProjectionNode projNode = (ProjectionNode) plan;
assertEquals(NodeType.GROUP_BY, projNode.getChild().getType());
- GroupbyNode groupByNode = (GroupbyNode) projNode.getChild();
+ GroupbyNode groupByNode = projNode.getChild();
assertEquals(NodeType.JOIN, groupByNode.getChild().getType());
- JoinNode joinNode = (JoinNode) groupByNode.getChild();
+ JoinNode joinNode = groupByNode.getChild();
assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
- ScanNode leftNode = (ScanNode) joinNode.getLeftChild();
- assertEquals("dept", leftNode.getTableId());
+ ScanNode leftNode = joinNode.getLeftChild();
+ assertEquals("dept", leftNode.getTableName());
assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
- ScanNode rightNode = (ScanNode) joinNode.getRightChild();
- assertEquals("score", rightNode.getTableId());
+ ScanNode rightNode = joinNode.getRightChild();
+ assertEquals("score", rightNode.getTableName());
}
@@ -453,9 +428,8 @@ public class TestLogicalPlanner {
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.STORE, root.getChild().getType());
- StoreTableNode storeNode = (StoreTableNode) root.getChild();
+ StoreTableNode storeNode = root.getChild();
testQuery7(storeNode.getChild());
- //LogicalOptimizer.optimize(context, plan);
}
@Test
@@ -469,20 +443,20 @@ public class TestLogicalPlanner {
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projNode = (ProjectionNode) root.getChild();
+ ProjectionNode projNode = root.getChild();
assertEquals(NodeType.SORT, projNode.getChild().getType());
- SortNode sortNode = (SortNode) projNode.getChild();
+ SortNode sortNode = projNode.getChild();
assertEquals(NodeType.JOIN, sortNode.getChild().getType());
- JoinNode joinNode = (JoinNode) sortNode.getChild();
+ JoinNode joinNode = sortNode.getChild();
assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
- ScanNode leftNode = (ScanNode) joinNode.getLeftChild();
- assertEquals("dept", leftNode.getTableId());
+ ScanNode leftNode = joinNode.getLeftChild();
+ assertEquals("dept", leftNode.getTableName());
assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
- ScanNode rightNode = (ScanNode) joinNode.getRightChild();
- assertEquals("score", rightNode.getTableId());
+ ScanNode rightNode = joinNode.getRightChild();
+ assertEquals("score", rightNode.getTableName());
}
@Test
@@ -496,10 +470,10 @@ public class TestLogicalPlanner {
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projNode = (ProjectionNode) root.getChild();
+ ProjectionNode projNode = root.getChild();
assertEquals(NodeType.LIMIT, projNode.getChild().getType());
- LimitNode limitNode = (LimitNode) projNode.getChild();
+ LimitNode limitNode = projNode.getChild();
assertEquals(NodeType.SORT, limitNode.getChild().getType());
}
@@ -514,21 +488,12 @@ public class TestLogicalPlanner {
assertEquals(NodeType.ROOT, plan.getType());
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projNode = (ProjectionNode) root.getChild();
+ ProjectionNode projNode = root.getChild();
assertEquals(NodeType.SELECTION, projNode.getChild().getType());
- SelectionNode selNode = (SelectionNode) projNode.getChild();
+ SelectionNode selNode = projNode.getChild();
assertEquals(NodeType.SCAN, selNode.getChild().getType());
- ScanNode scanNode = (ScanNode) selNode.getChild();
- assertEquals(scanNode.getTableId(), "employee");
-
- /*
- LogicalNode optimized = LogicalOptimizer.optimize(expr, plan);
- assertEquals(NodeType.ROOT, optimized.getType());
- root = (LogicalRootNode) optimized;
-
- assertEquals(NodeType.SCAN, root.getSubNode().getType());
- scanNode = (ScanNode) root.getSubNode();
- assertEquals("employee", scanNode.getTableId());*/
+ ScanNode scanNode = selNode.getChild();
+ assertEquals(scanNode.getTableName(), "employee");
}
@@ -625,7 +590,6 @@ public class TestLogicalPlanner {
expr = sqlAnalyzer.parse(ALIAS[1]);
plan = planner.createPlan(expr).getRootBlock().getRoot();
-// plan = LogicalOptimizer.optimize(expr, plan);
root = (LogicalRootNode) plan;
finalSchema = root.getOutSchema();
@@ -662,7 +626,7 @@ public class TestLogicalPlanner {
LogicalRootNode root = (LogicalRootNode) plan;
testJsonSerDerObject(root);
assertEquals(NodeType.CREATE_TABLE, root.getChild().getType());
- CreateTableNode createTable = (CreateTableNode) root.getChild();
+ CreateTableNode createTable = root.getChild();
Schema def = createTable.getSchema();
assertEquals("name", def.getColumn(0).getColumnName());
@@ -749,26 +713,26 @@ public class TestLogicalPlanner {
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projNode = (ProjectionNode) root.getChild();
+ ProjectionNode projNode = root.getChild();
assertEquals(NodeType.UNION, projNode.getChild().getType());
- UnionNode u0 = (UnionNode) projNode.getChild();
+ UnionNode u0 = projNode.getChild();
assertEquals(NodeType.GROUP_BY, u0.getLeftChild().getType());
assertEquals(NodeType.UNION, u0.getRightChild().getType());
- GroupbyNode grp = (GroupbyNode) u0.getLeftChild();
+ GroupbyNode grp = u0.getLeftChild();
cuboids.add(Sets.newHashSet(grp.getGroupingColumns()));
- UnionNode u1 = (UnionNode) u0.getRightChild();
+ UnionNode u1 = u0.getRightChild();
assertEquals(NodeType.GROUP_BY, u1.getLeftChild().getType());
assertEquals(NodeType.UNION, u1.getRightChild().getType());
- grp = (GroupbyNode) u1.getLeftChild();
+ grp = u1.getLeftChild();
cuboids.add(Sets.newHashSet(grp.getGroupingColumns()));
- UnionNode u2 = (UnionNode) u1.getRightChild();
+ UnionNode u2 = u1.getRightChild();
assertEquals(NodeType.GROUP_BY, u2.getLeftChild().getType());
- grp = (GroupbyNode) u2.getRightChild();
+ grp = u2.getRightChild();
cuboids.add(Sets.newHashSet(grp.getGroupingColumns()));
assertEquals(NodeType.GROUP_BY, u2.getRightChild().getType());
- grp = (GroupbyNode) u2.getLeftChild();
+ grp = u2.getLeftChild();
cuboids.add(Sets.newHashSet(grp.getGroupingColumns()));
assertEquals((int)Math.pow(2, 2), cuboids.size());
@@ -791,50 +755,48 @@ public class TestLogicalPlanner {
assertEquals(NodeType.ROOT, plan.getType());
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.UNION, root.getChild().getType());
- UnionNode union = (UnionNode) root.getChild();
- assertEquals(NodeType.PROJECTION, union.getLeftChild().getType());
- ProjectionNode projL = (ProjectionNode) union.getLeftChild();
- assertEquals(NodeType.SELECTION, projL.getChild().getType());
- assertEquals(NodeType.PROJECTION, union.getRightChild().getType());
- ProjectionNode projR = (ProjectionNode) union.getRightChild();
- assertEquals(NodeType.SELECTION, projR.getChild().getType());
+ UnionNode union = root.getChild();
+ assertEquals(NodeType.TABLE_SUBQUERY, union.getLeftChild().getType());
+ TableSubQueryNode leftSub = union.getLeftChild();
+ assertEquals(NodeType.PROJECTION, leftSub.getSubQuery().getType());
+ assertEquals(NodeType.TABLE_SUBQUERY, union.getRightChild().getType());
+ TableSubQueryNode rightSub = union.getRightChild();
+ assertEquals(NodeType.PROJECTION, rightSub.getSubQuery().getType());
}
@Test
public final void testSetPlan2() throws PlanningException {
// for testing multiple set statements
Expr expr = sqlAnalyzer.parse(setStatements[1]);
- LogicalNode plan = planner.createPlan(expr).getRootBlock().getRoot();
- testJsonSerDerObject(plan);
- assertEquals(NodeType.ROOT, plan.getType());
- LogicalRootNode root = (LogicalRootNode) plan;
+ LogicalPlan plan = planner.createPlan(expr);
+ System.out.println(plan);
+ LogicalRootNode root = plan.getRootBlock().getRoot();
+ testJsonSerDerObject(root);
+ assertEquals(NodeType.ROOT, root.getType());
assertEquals(NodeType.UNION, root.getChild().getType());
- UnionNode union = (UnionNode) root.getChild();
- assertEquals(NodeType.PROJECTION, union.getLeftChild().getType());
- assertEquals(NodeType.INTERSECT, union.getRightChild().getType());
- IntersectNode intersect = (IntersectNode) union.getRightChild();
- assertEquals(NodeType.PROJECTION, intersect.getLeftChild().getType());
- assertEquals(NodeType.PROJECTION, intersect.getRightChild().getType());
+ UnionNode union = root.getChild();
+ assertEquals(NodeType.TABLE_SUBQUERY, union.getLeftChild().getType());
+ assertEquals(NodeType.TABLE_SUBQUERY, union.getRightChild().getType());
+ TableSubQueryNode subQuery = union.getRightChild();
+ assertEquals(NodeType.INTERSECT, subQuery.getSubQuery().getType());
}
@Test
public final void testSetPlan3() throws PlanningException {
// for testing multiple set statements
Expr expr = sqlAnalyzer.parse(setStatements[2]);
- LogicalNode plan = planner.createPlan(expr).getRootBlock().getRoot();
- testJsonSerDerObject(plan);
- assertEquals(NodeType.ROOT, plan.getType());
- LogicalRootNode root = (LogicalRootNode) plan;
+ LogicalPlan plan = planner.createPlan(expr);
+ LogicalRootNode root = plan.getRootBlock().getRoot();
+ testJsonSerDerObject(root);
+ assertEquals(NodeType.ROOT, root.getType());
assertEquals(NodeType.EXCEPT, root.getChild().getType());
- ExceptNode except = (ExceptNode) root.getChild();
- assertEquals(NodeType.UNION, except.getLeftChild().getType());
- assertEquals(NodeType.INTERSECT, except.getRightChild().getType());
- UnionNode union = (UnionNode) except.getLeftChild();
- assertEquals(NodeType.PROJECTION, union.getLeftChild().getType());
- assertEquals(NodeType.PROJECTION, union.getRightChild().getType());
- IntersectNode intersect = (IntersectNode) except.getRightChild();
- assertEquals(NodeType.PROJECTION, intersect.getLeftChild().getType());
- assertEquals(NodeType.PROJECTION, intersect.getRightChild().getType());
+ ExceptNode except = root.getChild();
+ assertEquals(NodeType.TABLE_SUBQUERY, except.getLeftChild().getType());
+ assertEquals(NodeType.TABLE_SUBQUERY, except.getRightChild().getType());
+ TableSubQueryNode leftSubQuery = except.getLeftChild();
+ TableSubQueryNode rightSubQuery = except.getRightChild();
+ assertEquals(NodeType.UNION, leftSubQuery.getSubQuery().getType());
+ assertEquals(NodeType.INTERSECT, rightSubQuery.getSubQuery().getType());
}
static final String [] setQualifiers = {
@@ -851,7 +813,7 @@ public class TestLogicalPlanner {
assertEquals(NodeType.ROOT, plan.getType());
LogicalRootNode root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- ProjectionNode projectionNode = (ProjectionNode) root.getChild();
+ ProjectionNode projectionNode = root.getChild();
assertEquals(NodeType.SCAN, projectionNode.getChild().getType());
context = sqlAnalyzer.parse(setQualifiers[1]);
@@ -860,7 +822,7 @@ public class TestLogicalPlanner {
assertEquals(NodeType.ROOT, plan.getType());
root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- projectionNode = (ProjectionNode) root.getChild();
+ projectionNode = root.getChild();
assertEquals(NodeType.GROUP_BY, projectionNode.getChild().getType());
context = sqlAnalyzer.parse(setQualifiers[2]);
@@ -868,7 +830,7 @@ public class TestLogicalPlanner {
testJsonSerDerObject(plan);
root = (LogicalRootNode) plan;
assertEquals(NodeType.PROJECTION, root.getChild().getType());
- projectionNode = (ProjectionNode) root.getChild();
+ projectionNode = root.getChild();
assertEquals(NodeType.SCAN, projectionNode.getChild().getType());
}
@@ -962,8 +924,8 @@ public class TestLogicalPlanner {
}
private static InsertNode getInsertNode(LogicalPlan plan) {
- LogicalRootNode root = (LogicalRootNode) plan.getRootBlock().getRoot();
+ LogicalRootNode root = plan.getRootBlock().getRoot();
assertEquals(NodeType.INSERT, root.getChild().getType());
- return (InsertNode) root.getChild();
+ return root.getChild();
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
index 4821751..46fd648 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestPlannerUtil.java
@@ -110,27 +110,6 @@ public class TestPlannerUtil {
System.out.println(root);
}
- @Test
- public final void testTrasformTwoPhaseWithStore() throws PlanningException {
- Expr expr = analyzer.parse(TestLogicalPlanner.QUERIES[9]);
- LogicalNode plan = planner.createPlan(expr).getRootBlock().getRoot();
-
- assertEquals(NodeType.ROOT, plan.getType());
- UnaryNode unary = (UnaryNode) plan;
- assertEquals(NodeType.PROJECTION, unary.getChild().getType());
- ProjectionNode proj = (ProjectionNode) unary.getChild();
- assertEquals(NodeType.GROUP_BY, proj.getChild().getType());
- GroupbyNode groupby = (GroupbyNode) proj.getChild();
- unary = (UnaryNode) PlannerUtil.transformGroupbyTo2PWithStore(
- groupby, "test");
- assertEquals(NodeType.STORE, unary.getChild().getType());
- unary = (UnaryNode) unary.getChild();
-
- assertEquals(groupby.getInSchema(), unary.getOutSchema());
-
- assertEquals(NodeType.GROUP_BY, unary.getChild().getType());
- }
-
private final class TwoPhaseBuilder implements LogicalNodeVisitor {
@Override
public void visit(LogicalNode node) {
@@ -158,10 +137,10 @@ public class TestPlannerUtil {
assertEquals(NodeType.SCAN, joinNode.getLeftChild().getType());
ScanNode leftNode = (ScanNode) joinNode.getLeftChild();
- assertEquals("employee", leftNode.getTableId());
+ assertEquals("employee", leftNode.getTableName());
assertEquals(NodeType.SCAN, joinNode.getRightChild().getType());
ScanNode rightNode = (ScanNode) joinNode.getRightChild();
- assertEquals("dept", rightNode.getTableId());
+ assertEquals("dept", rightNode.getTableName());
LogicalNode node = PlannerUtil.findTopNode(root, NodeType.ROOT);
assertEquals(NodeType.ROOT, node.getType());
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
deleted file mode 100644
index 7572ad5..0000000
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestGlobalQueryOptimizer.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.planner.global;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.algebra.Expr;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.engine.eval.TestEvalTree.TestSum;
-import org.apache.tajo.engine.parser.SQLAnalyzer;
-import org.apache.tajo.engine.planner.LogicalOptimizer;
-import org.apache.tajo.engine.planner.LogicalPlan;
-import org.apache.tajo.engine.planner.LogicalPlanner;
-import org.apache.tajo.engine.planner.PlanningException;
-import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.master.ExecutionBlock;
-import org.apache.tajo.master.GlobalPlanner;
-import org.apache.tajo.storage.*;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestGlobalQueryOptimizer {
- private static TajoTestingCluster util;
- private static TajoConf conf;
- private static CatalogService catalog;
- private static GlobalPlanner planner;
- private static Schema schema;
- private static SQLAnalyzer analyzer;
- private static LogicalPlanner logicalPlanner;
- private static LogicalOptimizer logicalOptimizer;
- private static QueryId queryId;
- private static GlobalOptimizer optimizer;
-
- @BeforeClass
- public static void setup() throws Exception {
- util = new TajoTestingCluster();
- util.startCatalogCluster();
- int i, j;
-
- schema = new Schema();
- schema.addColumn("id", Type.INT4);
- schema.addColumn("age", Type.INT4);
- schema.addColumn("name", Type.TEXT);
- schema.addColumn("salary", Type.INT4);
-
- TableMeta meta;
-
- conf = new TajoConf(util.getConfiguration());
- catalog = util.getMiniCatalogCluster().getCatalog();
- AbstractStorageManager sm = StorageManagerFactory.getStorageManager(util.getConfiguration());
- FunctionDesc funcDesc = new FunctionDesc("sumtest", TestSum.class, FunctionType.GENERAL,
- CatalogUtil.newDataTypesWithoutLen(Type.INT4),
- CatalogUtil.newDataTypesWithoutLen(Type.INT4));
- catalog.registerFunction(funcDesc);
- FileSystem fs = sm.getFileSystem();
-
- AsyncDispatcher dispatcher = new AsyncDispatcher();
-
- planner = new GlobalPlanner(conf, sm,
- dispatcher.getEventHandler());
- analyzer = new SQLAnalyzer();
- logicalPlanner = new LogicalPlanner(catalog);
- logicalOptimizer = new LogicalOptimizer();
-
- int tbNum = 2;
- int tupleNum;
- Appender appender;
- Tuple t = new VTuple(4);
- t.put(new Datum[] {
- DatumFactory.createInt4(1), DatumFactory.createInt4(32),
- DatumFactory.createText("h"), DatumFactory.createInt4(10)});
-
- for (i = 0; i < tbNum; i++) {
- meta = CatalogUtil.newTableMeta((Schema) schema.clone(), StoreType.CSV);
- meta.putOption(CSVFile.DELIMITER, ",");
-
- Path dataRoot = sm.getBaseDir();
- Path tablePath = StorageUtil.concatPath(dataRoot, "table"+i, "file.csv");
- if (fs.exists(tablePath.getParent())) {
- fs.delete(tablePath.getParent(), true);
- }
- fs.mkdirs(tablePath.getParent());
- appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, tablePath);
- appender.init();
- tupleNum = 100;
- for (j = 0; j < tupleNum; j++) {
- appender.addTuple(t);
- }
- appender.close();
-
- TableDesc desc = CatalogUtil
- .newTableDesc("table" + i, (TableMeta) meta.clone(), sm.getTablePath("table" + i));
- catalog.addTable(desc);
- }
-
- //QueryIdFactory.reset();
- queryId = QueryIdFactory.newQueryId();
- optimizer = new GlobalOptimizer();
- }
-
- @AfterClass
- public static void terminate() throws IOException {
- util.shutdownCatalogCluster();
- }
-
- @Test
- public void testReduceLogicalQueryUnitSteps() throws IOException, PlanningException {
- Expr expr = analyzer.parse(
- "select table0.age,table0.salary,table1.salary from table0,table1 where table0.salary = table1.salary order by table0.age");
- LogicalPlan plan = logicalPlanner.createPlan(expr);
- LogicalNode rootNode = logicalOptimizer.optimize(plan);
-
- MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
- globalPlan = optimizer.optimize(globalPlan);
-
- ExecutionBlock unit = globalPlan.getRoot();
- StoreTableNode store = unit.getStoreTableNode();
- assertEquals(NodeType.PROJECTION, store.getChild().getType());
- ProjectionNode proj = (ProjectionNode) store.getChild();
- assertEquals(NodeType.SORT, proj.getChild().getType());
- SortNode sort = (SortNode) proj.getChild();
- assertEquals(NodeType.SCAN, sort.getChild().getType());
- ScanNode scan = (ScanNode) sort.getChild();
-
- assertTrue(unit.hasChildBlock());
- unit = unit.getChildBlock(scan);
- store = unit.getStoreTableNode();
- assertEquals(NodeType.SORT, store.getChild().getType());
- sort = (SortNode) store.getChild();
- assertEquals(NodeType.JOIN, sort.getChild().getType());
-
- assertTrue(unit.hasChildBlock());
- for (ScanNode prevscan : unit.getScanNodes()) {
- ExecutionBlock prev = unit.getChildBlock(prevscan);
- store = prev.getStoreTableNode();
- assertEquals(NodeType.SCAN, store.getChild().getType());
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java
new file mode 100644
index 0000000..cc5a001
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/global/TestMasterPlan.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.global;
+
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.ipc.TajoWorkerProtocol;
+import org.apache.tajo.master.ExecutionBlock;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestMasterPlan {
+
+ @Test
+ public void testConnect() {
+ MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
+
+ ExecutionBlock eb1 = masterPlan.newExecutionBlock();
+ ExecutionBlock eb2 = masterPlan.newExecutionBlock();
+ ExecutionBlock eb3 = masterPlan.newExecutionBlock();
+
+ masterPlan.addConnect(eb1, eb2, TajoWorkerProtocol.PartitionType.LIST_PARTITION);
+ assertTrue(masterPlan.isConnected(eb1.getId(), eb2.getId()));
+ assertTrue(masterPlan.isReverseConnected(eb2.getId(), eb1.getId()));
+
+ masterPlan.addConnect(eb3, eb2, TajoWorkerProtocol.PartitionType.LIST_PARTITION);
+ assertTrue(masterPlan.isConnected(eb1.getId(), eb2.getId()));
+ assertTrue(masterPlan.isConnected(eb3.getId(), eb2.getId()));
+
+ assertTrue(masterPlan.isReverseConnected(eb2.getId(), eb1.getId()));
+ assertTrue(masterPlan.isReverseConnected(eb2.getId(), eb3.getId()));
+
+ masterPlan.disconnect(eb3, eb2);
+ assertFalse(masterPlan.isConnected(eb3, eb2));
+ assertFalse(masterPlan.isReverseConnected(eb2, eb3));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
index b714981..beec1ea 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBNLJoinExec.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -140,7 +141,7 @@ public class TestBNLJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testBNLCrossJoin");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), merged, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
Expr expr = analyzer.parse(QUERIES[0]);
LogicalNode plan = planner.createPlan(expr).getRootBlock().getRoot();
@@ -175,7 +176,7 @@ public class TestBNLJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testBNLInnerJoin");
TaskAttemptContext ctx =
- new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(),
merged, workDir);
Expr context = analyzer.parse(QUERIES[1]);
LogicalNode plan = planner.createPlan(context).getRootBlock().getRoot();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
index 8021882..1fc82c0 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestBSTIndexExec.java
@@ -21,6 +21,7 @@ package org.apache.tajo.engine.planner.physical;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -40,7 +41,6 @@ import org.apache.tajo.engine.planner.logical.ScanNode;
import org.apache.tajo.storage.*;
import org.apache.tajo.storage.index.bst.BSTIndex;
import org.apache.tajo.util.CommonTestingUtil;
-import org.apache.tajo.util.TUtil;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -94,7 +94,7 @@ public class TestBSTIndexExec {
this.idxSchema = new Schema();
idxSchema.addColumn("managerId", Type.INT4);
SortSpec[] sortKeys = new SortSpec[1];
- sortKeys[0] = new SortSpec(idxSchema.getColumn("managerId"), true, false);
+ sortKeys[0] = new SortSpec(idxSchema.getColumnByFQN("managerId"), true, false);
this.comp = new TupleComparator(idxSchema, sortKeys);
this.writer = new BSTIndex(conf).getIndexWriter(idxPath,
@@ -159,7 +159,7 @@ public class TestBSTIndexExec {
Fragment[] frags = StorageManager.splitNG(conf, "employee", meta, tablePath, Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEqual");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), new Fragment[] { frags[0] }, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), new Fragment[] { frags[0] }, workDir);
Expr expr = analyzer.parse(QUERY);
LogicalPlan plan = planner.createPlan(expr);
LogicalNode rootNode = optimizer.optimize(plan);
@@ -189,10 +189,10 @@ public class TestBSTIndexExec {
@Override
public PhysicalExec createScanPlan(TaskAttemptContext ctx, ScanNode scanNode)
throws IOException {
- Preconditions.checkNotNull(ctx.getTable(scanNode.getTableId()),
- "Error: There is no table matched to %s", scanNode.getTableId());
+ Preconditions.checkNotNull(ctx.getTable(scanNode.getTableName()),
+ "Error: There is no table matched to %s", scanNode.getTableName());
- Fragment[] fragments = ctx.getTables(scanNode.getTableId());
+ Fragment[] fragments = ctx.getTables(scanNode.getTableName());
Datum[] datum = new Datum[]{DatumFactory.createInt4(rndKey)};
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
index 01fd370..864c776 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -33,7 +34,6 @@ import org.apache.tajo.engine.planner.*;
import org.apache.tajo.engine.planner.logical.LogicalNode;
import org.apache.tajo.storage.*;
import org.apache.tajo.util.CommonTestingUtil;
-import org.apache.tajo.util.TUtil;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -112,7 +112,7 @@ public class TestExternalSortExec {
Integer.MAX_VALUE);
Path workDir = new Path(testDir, TestExternalSortExec.class.getName());
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), new Fragment[] { frags[0] }, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), new Fragment[] { frags[0] }, workDir);
Expr expr = analyzer.parse(QUERIES[0]);
LogicalPlan plan = planner.createPlan(expr);
LogicalNode rootNode = plan.getRootBlock().getRoot();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
index 886dddc..94de6bc 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashAntiJoinExec.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -147,7 +148,7 @@ public class TestHashAntiJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashAntiJoin");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), merged, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
Expr expr = analyzer.parse(QUERIES[0]);
LogicalPlan plan = planner.createPlan(expr);
optimizer.optimize(plan);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
index cf89cf8..0654042 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashJoinExec.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -140,7 +141,7 @@ public class TestHashJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashInnerJoin");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), merged, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
Expr expr = analyzer.parse(QUERIES[0]);
LogicalNode plan = planner.createPlan(expr).getRootBlock().getRoot();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
index d986a8f..1d0d139 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestHashSemiJoinExec.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -151,7 +152,7 @@ public class TestHashSemiJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testHashSemiJoin");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), merged, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
Expr expr = analyzer.parse(QUERIES[0]);
LogicalPlan plan = planner.createPlan(expr);
optimizer.optimize(plan);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
index e77a734..f130187 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestMergeJoinExec.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -156,7 +157,7 @@ public class TestMergeJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testMergeInnerJoin");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), merged, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
Expr expr = analyzer.parse(QUERIES[0]);
LogicalNode plan = planner.createPlan(expr).getRootBlock().getRoot();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
index 2d82f6c..5b2f658 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestNLJoinExec.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -33,6 +34,7 @@ import org.apache.tajo.engine.planner.LogicalPlanner;
import org.apache.tajo.engine.planner.PhysicalPlanner;
import org.apache.tajo.engine.planner.PhysicalPlannerImpl;
import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.planner.logical.LogicalNode;
import org.apache.tajo.storage.*;
import org.apache.tajo.util.CommonTestingUtil;
@@ -59,6 +61,8 @@ public class TestNLJoinExec {
private TableDesc employee;
private TableDesc people;
+ private MasterPlan masterPlan;
+
@Before
public void setUp() throws Exception {
util = new TajoTestingCluster();
@@ -118,6 +122,8 @@ public class TestNLJoinExec {
catalog.addTable(people);
analyzer = new SQLAnalyzer();
planner = new LogicalPlanner(catalog);
+
+ masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
}
@After
@@ -142,7 +148,7 @@ public class TestNLJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testNLCrossJoin");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), merged, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(), merged, workDir);
Expr context = analyzer.parse(QUERIES[0]);
LogicalNode plan = planner.createPlan(context).getRootBlock().getRoot();
@@ -170,7 +176,7 @@ public class TestNLJoinExec {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testNLInnerJoin");
TaskAttemptContext ctx = new TaskAttemptContext(conf,
- TUtil.newQueryUnitAttemptId(), merged, workDir);
+ LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan), merged, workDir);
Expr context = analyzer.parse(QUERIES[1]);
LogicalNode plan = planner.createPlan(context).getRootBlock().getRoot();
//LogicalOptimizer.optimize(ctx, plan);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
index 5358d3a..aa8e4d7 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestPhysicalPlanner.java
@@ -25,9 +25,7 @@ import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.tajo.QueryUnitAttemptId;
-import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.TaskAttemptContext;
+import org.apache.tajo.*;
import org.apache.tajo.algebra.Expr;
import org.apache.tajo.catalog.*;
import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
@@ -41,8 +39,8 @@ import org.apache.tajo.engine.eval.EvalNode;
import org.apache.tajo.engine.eval.EvalTreeUtil;
import org.apache.tajo.engine.parser.SQLAnalyzer;
import org.apache.tajo.engine.planner.*;
+import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
import org.apache.tajo.master.TajoMaster;
import org.apache.tajo.storage.*;
import org.apache.tajo.storage.index.bst.BSTIndex;
@@ -60,6 +58,7 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType;
import static org.junit.Assert.*;
public class TestPhysicalPlanner {
@@ -75,6 +74,8 @@ public class TestPhysicalPlanner {
private static TableDesc employee = null;
private static TableDesc score = null;
+ private static MasterPlan masterPlan;
+
@BeforeClass
public static void setUp() throws Exception {
util = new TajoTestingCluster();
@@ -148,6 +149,8 @@ public class TestPhysicalPlanner {
analyzer = new SQLAnalyzer();
planner = new LogicalPlanner(catalog);
optimizer = new LogicalOptimizer();
+
+ masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), null, null);
}
@AfterClass
@@ -180,8 +183,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "employee", employee.getMeta(),
employee.getPath(), Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanPlan");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil
- .newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr expr = analyzer.parse(QUERIES[0]);
LogicalPlan plan = planner.createPlan(expr);
@@ -210,8 +212,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "employee", employee.getMeta(),
employee.getPath(), Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateScanWithFilterPlan");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil
- .newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr expr = analyzer.parse(QUERIES[16]);
LogicalPlan plan = planner.createPlan(expr);
@@ -238,7 +239,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByPlan");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr context = analyzer.parse(QUERIES[7]);
LogicalPlan plan = planner.createPlan(context);
@@ -268,7 +269,7 @@ public class TestPhysicalPlanner {
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir(
"target/test-data/testHashGroupByPlanWithALLField");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr expr = analyzer.parse(QUERIES[15]);
LogicalPlan plan = planner.createPlan(expr);
@@ -296,7 +297,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testSortGroupByPlan");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[]{frags[0]}, workDir);
Expr context = analyzer.parse(QUERIES[7]);
LogicalPlan plan = planner.createPlan(context);
@@ -353,7 +354,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlan");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] },
workDir);
ctx.setOutputPath(new Path(workDir, "grouped1"));
@@ -394,7 +395,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testStorePlanWithRCFile");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] },
workDir);
ctx.setOutputPath(new Path(workDir, "grouped2"));
@@ -433,10 +434,9 @@ public class TestPhysicalPlanner {
public final void testPartitionedStorePlan() throws IOException, PlanningException {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
- QueryUnitAttemptId id = TUtil.newQueryUnitAttemptId();
+ QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testPartitionedStorePlan");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, id, new Fragment[] { frags[0] },
- workDir);
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, id, new Fragment[] { frags[0] }, workDir);
Expr context = analyzer.parse(QUERIES[7]);
LogicalPlan plan = planner.createPlan(context);
LogicalNode rootNode = plan.getRootBlock().getRoot();
@@ -444,13 +444,13 @@ public class TestPhysicalPlanner {
int numPartitions = 3;
Column key1 = new Column("score.deptName", Type.TEXT);
Column key2 = new Column("score.class", Type.TEXT);
- StoreTableNode storeNode = new StoreTableNode("partition");
- storeNode.setPartitions(PartitionType.HASH, new Column[]{key1, key2}, numPartitions);
- PlannerUtil.insertNode(rootNode, storeNode);
+ DataChannel dataChannel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
+ PartitionType.HASH_PARTITION, numPartitions);
+ dataChannel.setPartitionKey(new Column[]{key1, key2});
+ ctx.setDataChannel(dataChannel);
rootNode = optimizer.optimize(plan);
- TableMeta outputMeta = CatalogUtil.newTableMeta(rootNode.getOutSchema(),
- StoreType.CSV);
+ TableMeta outputMeta = CatalogUtil.newTableMeta(rootNode.getOutSchema(), StoreType.CSV);
FileSystem fs = sm.getFileSystem();
@@ -492,7 +492,7 @@ public class TestPhysicalPlanner {
throws IOException, PlanningException {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
- QueryUnitAttemptId id = TUtil.newQueryUnitAttemptId();
+ QueryUnitAttemptId id = LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan);
Path workDir = CommonTestingUtil.getTestDir(
"target/test-data/testPartitionedStorePlanWithEmptyGroupingSet");
@@ -502,9 +502,10 @@ public class TestPhysicalPlanner {
LogicalPlan plan = planner.createPlan(expr);
LogicalNode rootNode = plan.getRootBlock().getRoot();
int numPartitions = 1;
- StoreTableNode storeNode = new StoreTableNode("emptyset");
- storeNode.setPartitions(PartitionType.HASH, new Column[] {}, numPartitions);
- PlannerUtil.insertNode(rootNode, storeNode);
+ DataChannel dataChannel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
+ PartitionType.HASH_PARTITION, numPartitions);
+ dataChannel.setPartitionKey(new Column[]{});
+ ctx.setDataChannel(dataChannel);
optimizer.optimize(plan);
TableMeta outputMeta = CatalogUtil.newTableMeta(rootNode.getOutSchema(),
@@ -549,14 +550,14 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testAggregationFunction");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr context = analyzer.parse(QUERIES[8]);
LogicalPlan plan = planner.createPlan(context);
LogicalNode rootNode = optimizer.optimize(plan);
// Set all aggregation functions to the first phase mode
- GroupbyNode groupbyNode = (GroupbyNode) PlannerUtil.findTopNode(rootNode, NodeType.GROUP_BY);
+ GroupbyNode groupbyNode = PlannerUtil.findTopNode(rootNode, NodeType.GROUP_BY);
for (Target target : groupbyNode.getTargets()) {
for (EvalNode eval : EvalTreeUtil.findDistinctAggFunction(target.getEvalTree())) {
if (eval instanceof AggFuncCallEval) {
@@ -582,7 +583,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCountFunction");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr context = analyzer.parse(QUERIES[9]);
LogicalPlan plan = planner.createPlan(context);
@@ -613,7 +614,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "score", score.getMeta(), score.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testGroupByWithNullValue");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr context = analyzer.parse(QUERIES[11]);
LogicalPlan plan = planner.createPlan(context);
@@ -636,7 +637,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testUnionPlan");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { frags[0] }, workDir);
Expr context = analyzer.parse(QUERIES[0]);
LogicalPlan plan = planner.createPlan(context);
@@ -660,7 +661,7 @@ public class TestPhysicalPlanner {
@Test
public final void testEvalExpr() throws IOException, PlanningException {
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testEvalExpr");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] { }, workDir);
Expr expr = analyzer.parse(QUERIES[12]);
LogicalPlan plan = planner.createPlan(expr);
@@ -696,7 +697,7 @@ public class TestPhysicalPlanner {
Fragment[] frags = StorageManager.splitNG(conf, "employee", employee.getMeta(), employee.getPath(),
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testCreateIndex");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] {frags[0]}, workDir);
Expr context = analyzer.parse(createIndexStmt[0]);
LogicalPlan plan = planner.createPlan(context);
@@ -723,7 +724,7 @@ public class TestPhysicalPlanner {
Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testDuplicateEliminate");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] {frags[0]}, workDir);
Expr expr = analyzer.parse(duplicateElimination[0]);
LogicalPlan plan = planner.createPlan(expr);
@@ -755,23 +756,28 @@ public class TestPhysicalPlanner {
employee.getPath(), Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/testIndexedStoreExec");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil.newQueryUnitAttemptId(),
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility.newQueryUnitAttemptId(masterPlan),
new Fragment[] {frags[0]}, workDir);
Expr context = analyzer.parse(SORT_QUERY[0]);
LogicalPlan plan = planner.createPlan(context);
LogicalNode rootNode = optimizer.optimize(plan);
+ SortNode sortNode = PlannerUtil.findTopNode(rootNode, NodeType.SORT);
+ DataChannel channel = new DataChannel(masterPlan.newExecutionBlockId(), masterPlan.newExecutionBlockId(),
+ PartitionType.RANGE_PARTITION);
+ channel.setPartitionKey(PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys()).toArray());
+ ctx.setDataChannel(channel);
+
PhysicalPlanner phyPlanner = new PhysicalPlannerImpl(conf,sm);
PhysicalExec exec = phyPlanner.createPlan(ctx, rootNode);
- ProjectionExec proj = (ProjectionExec) exec;
- ExternalSortExec sort = (ExternalSortExec) proj.getChild();
-
- SortSpec[] sortSpecs = sort.getPlan().getSortKeys();
- IndexedStoreExec idxStoreExec = new IndexedStoreExec(ctx, sm, sort, sort.getSchema(), sort.getSchema(), sortSpecs);
+// ProjectionExec proj = (ProjectionExec) exec;
+// ExternalSortExec sort = (ExternalSortExec) proj.getChild();
+//
+// SortSpec[] sortSpecs = sort.getPlan().getSortKeys();
+ //IndexedStoreExec idxStoreExec = new IndexedStoreExec(ctx, sm, sort, sort.getSchema(), sort.getSchema(), sortSpecs);
Tuple tuple;
- exec = idxStoreExec;
exec.init();
exec.next();
exec.close();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
index 06c5bb7..c655e05 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/physical/TestSortExec.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.physical;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.algebra.Expr;
@@ -33,7 +34,6 @@ import org.apache.tajo.engine.planner.*;
import org.apache.tajo.engine.planner.logical.LogicalNode;
import org.apache.tajo.storage.*;
import org.apache.tajo.util.CommonTestingUtil;
-import org.apache.tajo.util.TUtil;
import org.junit.After;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -109,7 +109,7 @@ public class TestSortExec {
public final void testNext() throws IOException, PlanningException {
Fragment [] frags = sm.splitNG(conf, "employee", employeeMeta, tablePath, Integer.MAX_VALUE);
Path workDir = CommonTestingUtil.getTestDir("target/test-data/TestSortExec");
- TaskAttemptContext ctx = new TaskAttemptContext(conf, TUtil
+ TaskAttemptContext ctx = new TaskAttemptContext(conf, LocalTajoTestingUtility
.newQueryUnitAttemptId(),
new Fragment[] { frags[0] }, workDir);
Expr context = analyzer.parse(QUERIES[0]);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
index cc497a8..a11cdbf 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java
@@ -18,6 +18,7 @@
package org.apache.tajo.engine.query;
+import org.apache.tajo.client.ResultSetUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -61,6 +62,25 @@ public class TestGroupByQuery {
}
@Test
+ public final void testComplexParameterWithSubQuery() throws Exception {
+
+
+ ResultSet res = tpch.execute(
+ "select count(*) as total from ("+
+ " select * from lineitem " +
+ " union all"+
+ " select * from lineitem ) l");
+ try {
+ assertNotNull(res);
+ assertTrue(res.next());
+ assertTrue(10 == (int) res.getDouble("total"));
+ assertFalse(res.next());
+ } finally {
+ res.close();
+ }
+ }
+
+ @Test
public final void testComplexParameter2() throws Exception {
ResultSet res = tpch.execute("select count(*) + max(l_orderkey) as merged from lineitem");
try {
@@ -71,6 +91,8 @@ public class TestGroupByQuery {
}
}
+
+
//@Test
public final void testCube() throws Exception {
ResultSet res = tpch.execute(
@@ -85,19 +107,4 @@ public class TestGroupByQuery {
res.close();
}
}
-
- //@Test
- // TODO - to fix the limit processing and then enable it
- public final void testGroupByLimit() throws Exception {
- ResultSet res = tpch.execute("select l_orderkey from lineitem limit 2");
- try {
- int count = 0;
- for (;res.next();) {
- count++;
- }
- assertEquals(2, count);
- } finally {
- res.close();
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
index d006679..0182067 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestResultSetImpl.java
@@ -103,7 +103,7 @@ public class TestResultSetImpl {
assertEquals(schema.getColumnNum(), meta.getColumnCount());
for (int i = 0; i < meta.getColumnCount(); i++) {
assertEquals(schema.getColumn(i).getColumnName(), meta.getColumnName(i + 1));
- assertEquals(schema.getColumn(i).getTableName(), meta.getTableName(i + 1));
+ assertEquals(schema.getColumn(i).getQualifier(), meta.getTableName(i + 1));
assertEquals(schema.getColumn(i).getDataType().getClass().getCanonicalName(),
meta.getColumnTypeName(i + 1));
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
index 6c10888..df6eaa0 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java
@@ -400,9 +400,11 @@ public class TestSelectQuery {
for (;res.next();) {
count++;
}
+ assertEquals(10, count);
} finally {
res.close();
}
+
}
@Test
@@ -417,8 +419,7 @@ public class TestSelectQuery {
assertEquals(5, orderKeys.getMeta().getStat().getNumRows().intValue());
}
- //@Test
- // TODO - fix and enable this unit test
+ @Test
public final void testLimit() throws Exception {
ResultSet res = tpch.execute("select l_orderkey from lineitem limit 3");
try {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java
new file mode 100644
index 0000000..e445089
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestTableSubQuery.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.query;
+
+import org.apache.tajo.IntegrationTest;
+import org.apache.tajo.TpchTestBase;
+import org.apache.tajo.client.ResultSetUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import java.io.IOException;
+import java.sql.ResultSet;
+
+import static org.junit.Assert.*;
+
+@Category(IntegrationTest.class)
+public class TestTableSubQuery {
+ private static TpchTestBase tpch;
+ public TestTableSubQuery() throws IOException {
+ super();
+ }
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ tpch = TpchTestBase.getInstance();
+ }
+
+ @Test
+ public final void testTableSubquery1() throws Exception {
+ ResultSet res = tpch.execute(
+ "select l_orderkey from (select * from lineitem) as l");
+ try {
+ int count = 0;
+ for (;res.next();) {
+ count++;
+ }
+ assertEquals(5, count);
+ } finally {
+ res.close();
+ }
+ }
+
+ @Test
+ public final void testGroupBySubQuery() throws Exception {
+ ResultSet res = tpch.execute(
+ "select sum(l_extendedprice * l_discount) as revenue from (select * from lineitem) as l");
+ try {
+ assertNotNull(res);
+ assertTrue(res.next());
+ assertTrue(12908 == (int) res.getDouble("revenue"));
+ assertFalse(res.next());
+ } finally {
+ res.close();
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
index b5ce437..6ffc532 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
@@ -16,7 +16,7 @@ package org.apache.tajo.master;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.tajo.QueryIdFactory;
+import org.apache.tajo.LocalTajoTestingUtility;
import org.apache.tajo.TajoTestingCluster;
import org.apache.tajo.algebra.Expr;
import org.apache.tajo.benchmark.TPCH;
@@ -73,7 +73,7 @@ public class TestExecutionBlockCursor {
AsyncDispatcher dispatcher = new AsyncDispatcher();
dispatcher.init(conf);
dispatcher.start();
- planner = new GlobalPlanner(conf, sm, dispatcher.getEventHandler());
+ planner = new GlobalPlanner(conf, sm);
}
public static void tearDown() {
@@ -90,8 +90,10 @@ public class TestExecutionBlockCursor {
"join partsupp on s_suppkey = ps_suppkey " +
"join part on p_partkey = ps_partkey and p_type like '%BRASS' and p_size = 15");
LogicalPlan logicalPlan = logicalPlanner.createPlan(context);
- LogicalNode rootNode = optimizer.optimize(logicalPlan);
- MasterPlan plan = planner.build(QueryIdFactory.newQueryId(), (LogicalRootNode) rootNode);
+ optimizer.optimize(logicalPlan);
+ QueryContext queryContext = new QueryContext();
+ MasterPlan plan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), queryContext, logicalPlan);
+ planner.build(plan);
ExecutionBlockCursor cursor = new ExecutionBlockCursor(plan);
@@ -101,7 +103,7 @@ public class TestExecutionBlockCursor {
count++;
}
- // 4 input relations, 4 join, and 1 projection = 9 execution blocks
- assertEquals(9, count);
+ // 4 input relations, 4 join, and 1 terminal = 9 execution blocks
+ assertEquals(10, count);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java
new file mode 100644
index 0000000..1e89ce5
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/master/TestGlobalPlanner.java
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.master;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.LocalTajoTestingUtility;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.LogicalOptimizer;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.LogicalPlanner;
+import org.apache.tajo.engine.planner.PlanningException;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.storage.StorageManagerFactory;
+import org.apache.tajo.util.FileUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+
+public class TestGlobalPlanner {
+
+ private static TajoTestingCluster util;
+ private static CatalogService catalog;
+ private static SQLAnalyzer sqlAnalyzer;
+ private static LogicalPlanner planner;
+ private static LogicalOptimizer optimizer;
+ private static TPCH tpch;
+ private static GlobalPlanner globalPlanner;
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ util = new TajoTestingCluster();
+ util.startCatalogCluster();
+ catalog = util.getMiniCatalogCluster().getCatalog();
+ for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+ catalog.registerFunction(funcDesc);
+ }
+
+ // TPC-H Schema for Complex Queries
+ String [] tpchTables = {
+ "part", "supplier", "partsupp", "nation", "region", "lineitem", "orders", "customer"
+ };
+ tpch = new TPCH();
+ tpch.loadSchemas();
+ tpch.loadOutSchema();
+ for (String table : tpchTables) {
+ TableMeta m = CatalogUtil.newTableMeta(tpch.getSchema(table), CatalogProtos.StoreType.CSV);
+ TableDesc d = CatalogUtil.newTableDesc(table, m, new Path("file:///"));
+ catalog.addTable(d);
+ }
+
+ sqlAnalyzer = new SQLAnalyzer();
+ planner = new LogicalPlanner(catalog);
+ optimizer = new LogicalOptimizer();
+ globalPlanner = new GlobalPlanner(util.getConfiguration(),
+ StorageManagerFactory.getStorageManager(util.getConfiguration()));
+ }
+
+ private MasterPlan buildPlan(String sql) throws PlanningException, IOException {
+ Expr expr = sqlAnalyzer.parse(sql);
+ LogicalPlan plan = planner.createPlan(expr);
+ optimizer.optimize(plan);
+ QueryContext context = new QueryContext();
+ MasterPlan masterPlan = new MasterPlan(LocalTajoTestingUtility.newQueryId(), context, plan);
+ globalPlanner.build(masterPlan);
+ return masterPlan;
+ }
+
+ @Test
+ public void testSelectDistinct() throws Exception {
+ MasterPlan masterPlan = buildPlan("select distinct l_orderkey from lineitem");
+ }
+
+ @Test
+ public void testSortAfterGroupBy() throws Exception {
+ MasterPlan masterPlan = buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity");
+ }
+
+ @Test
+ public void testSortLimit() throws Exception {
+ MasterPlan masterPlan = buildPlan("select max(l_quantity) as max_quantity, l_orderkey from lineitem group by l_orderkey order by max_quantity limit 3");
+ }
+
+ @Test
+ public void testJoin() throws Exception {
+ buildPlan("select n_name, r_name, n_regionkey, r_regionkey from nation, region");
+ }
+
+ @Test
+ public void testMultipleJoin() throws Exception {
+ buildPlan(FileUtil.readTextFile(new File("src/test/queries/tpch_q2_simplified.tql")));
+ }
+
+ @Test
+ public void testUnion() throws IOException, PlanningException {
+ buildPlan("select o_custkey as num from orders union select c_custkey as num from customer union select p_partkey as num from part");
+ }
+
+ @Test
+ public void testSubQuery() throws IOException, PlanningException {
+ buildPlan("select l.l_orderkey from (select * from lineitem) l");
+ }
+
+ @Test
+ public void testSubQueryJoin() throws IOException, PlanningException {
+ buildPlan("select l.l_orderkey from (select * from lineitem) l join (select * from orders) o on l.l_orderkey = o.o_orderkey");
+ }
+
+ @Test
+ public void testSubQueryGroupBy() throws IOException, PlanningException {
+ buildPlan("select sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l");
+ }
+
+ @Test
+ public void testSubQueryGroupBy2() throws IOException, PlanningException {
+ buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey");
+ }
+
+ @Test
+ public void testSubQuerySortAfterGroup() throws IOException, PlanningException {
+ buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (select * from lineitem) as l group by l_orderkey order by l_orderkey");
+ }
+
+ @Test
+ public void testSubQuerySortAfterGroupMultiBlocks() throws IOException, PlanningException {
+ buildPlan(
+ "select l_orderkey, revenue from (" +
+ "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey"
+ +") l1"
+
+ );
+ }
+
+ @Test
+ public void testSubQuerySortAfterGroupMultiBlocks2() throws IOException, PlanningException {
+ buildPlan(
+ "select l_orderkey, revenue from (" +
+ "select l_orderkey, revenue from (" +
+ "select l_orderkey, sum(l_extendedprice*l_discount) as revenue from lineitem group by l_orderkey"
+ +") l1" +
+ ") l2 order by l_orderkey"
+
+ );
+ }
+
+ @Test
+ public void testComplexUnion1() throws Exception {
+ buildPlan(FileUtil.readTextFile(new File("src/test/queries/complex_union_1.sql")));
+ }
+
+ @Test
+ public void testComplexUnion2() throws Exception {
+ buildPlan(FileUtil.readTextFile(new File("src/test/queries/complex_union_2.sql")));
+ }
+
+ @Test
+ public void testUnionGroupBy1() throws Exception {
+ buildPlan("select l_orderkey, sum(l_extendedprice*l_discount) as revenue from (" +
+ "select * from lineitem " +
+ "union " +
+ "select * from lineitem ) l group by l_orderkey");
+ }
+}
[3/7] TAJO-184: Refactor GlobalPlanner and global plan data
structure. (hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
index cc5f848..bc2b3eb 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Repartitioner.java
@@ -18,27 +18,33 @@
package org.apache.tajo.master.querymaster;
+import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.catalog.*;
import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.catalog.statistics.StatisticsUtil;
import org.apache.tajo.catalog.statistics.TableStat;
import org.apache.tajo.conf.TajoConf.ConfVars;
import org.apache.tajo.engine.planner.PlannerUtil;
import org.apache.tajo.engine.planner.RangePartitionAlgorithm;
import org.apache.tajo.engine.planner.UniformRangePartition;
+import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.planner.logical.*;
import org.apache.tajo.engine.utils.TupleUtil;
import org.apache.tajo.exception.InternalException;
import org.apache.tajo.master.ExecutionBlock;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
import org.apache.tajo.master.querymaster.QueryUnit.IntermediateEntry;
+import org.apache.tajo.storage.AbstractStorageManager;
import org.apache.tajo.storage.Fragment;
import org.apache.tajo.storage.TupleRange;
import org.apache.tajo.util.TUtil;
+import org.apache.tajo.util.TajoIdUtils;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
@@ -47,6 +53,10 @@ import java.net.URI;
import java.util.*;
import java.util.Map.Entry;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType.HASH_PARTITION;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType.RANGE_PARTITION;
+
/**
* Repartitioner creates non-leaf tasks and shuffles intermediate data.
* It supports two repartition methods, such as hash and range repartition.
@@ -58,53 +68,68 @@ public class Repartitioner {
public static QueryUnit[] createJoinTasks(SubQuery subQuery)
throws IOException {
+ MasterPlan masterPlan = subQuery.getMasterPlan();
ExecutionBlock execBlock = subQuery.getBlock();
- //CatalogService catalog = subQuery.getContext().getCatalog();
+ QueryMasterTask.QueryMasterTaskContext masterContext = subQuery.getContext();
+ AbstractStorageManager storageManager = subQuery.getStorageManager();
ScanNode[] scans = execBlock.getScanNodes();
+ ExecutionBlock [] childBlocks = new ExecutionBlock[2];
+ childBlocks[0] = masterPlan.getChild(execBlock.getId(), 0);
+ childBlocks[1] = masterPlan.getChild(execBlock.getId(), 1);
+
Path tablePath;
Fragment [] fragments = new Fragment[2];
TableStat [] stats = new TableStat[2];
// initialize variables from the child operators
for (int i =0; i < 2; i++) {
- // TODO - temporarily tables should be stored in temporarily catalog for each query
- TableDesc tableDesc = subQuery.getContext().getTableDescMap().get(scans[i].getFromTable().getTableName());
- if (scans[i].getTableId().startsWith(ExecutionBlockId.EB_ID_PREFIX)) {
- tablePath = subQuery.getStorageManager().getTablePath(scans[i].getTableId());
- stats[i] = subQuery.getChildQuery(scans[i]).getTableStat();
+ TableDesc tableDesc = masterContext.getTableDescMap().get(scans[i].getFromTable().getTableName());
+ if (tableDesc == null) { // if it is a real table stored on storage
+ // TODO - to be fixed (wrong directory)
+ tablePath = storageManager.getTablePath(scans[i].getTableName());
+ stats[i] = masterContext.getSubQuery(childBlocks[i].getId()).getTableStat();
+ fragments[i] = new Fragment(scans[i].getTableName(), tablePath,
+ CatalogUtil.newTableMeta(scans[i].getInSchema(), StoreType.CSV), 0, 0);
} else {
tablePath = tableDesc.getPath();
stats[i] = tableDesc.getMeta().getStat();
- }
-
- if (scans[i].isLocal()) { // it only requires a dummy fragment.
- fragments[i] = new Fragment(scans[i].getTableId(), tablePath,
- CatalogUtil.newTableMeta(scans[i].getInSchema(), StoreType.CSV), 0, 0);
- } else {
- fragments[i] = subQuery.getStorageManager().getSplits(scans[i].getTableId(),
- tableDesc.getMeta(), tablePath).get(0);
+ fragments[i] = storageManager.getSplits(scans[i].getTableName(),
+ tableDesc.getMeta(), tablePath).get(0);
}
}
// Assigning either fragments or fetch urls to query units
QueryUnit [] tasks;
- if (scans[0].isBroadcast() || scans[1].isBroadcast()) {
+ boolean leftSmall = execBlock.isBroadcastTable(scans[0].getCanonicalName());
+ boolean rightSmall = execBlock.isBroadcastTable(scans[1].getCanonicalName());
+
+ if (leftSmall && rightSmall) {
+ LOG.info("[Distributed Join Strategy] : Immediate Two Way Join on Single Machine");
tasks = new QueryUnit[1];
tasks[0] = new QueryUnit(QueryIdFactory.newQueryUnitId(subQuery.getId(), 0),
false, subQuery.getEventHandler());
tasks[0].setLogicalPlan(execBlock.getPlan());
- tasks[0].setFragment(scans[0].getTableId(), fragments[0]);
- tasks[0].setFragment(scans[1].getTableId(), fragments[1]);
+ tasks[0].setFragment(scans[0].getTableName(), fragments[0]);
+ tasks[0].setFragment(scans[1].getTableName(), fragments[1]);
+ } else if (leftSmall ^ rightSmall) {
+ LOG.info("[Distributed Join Strategy] : Broadcast Join");
+ int broadcastIdx = leftSmall ? 0 : 1;
+ int baseScanIdx = leftSmall ? 1 : 0;
+
+ LOG.info("Broadcasting Table Volume: " + stats[broadcastIdx].getNumBytes());
+ LOG.info("Base Table Volume: " + stats[baseScanIdx].getNumBytes());
+
+ tasks = createLeafTasksWithBroadcastTable(subQuery, baseScanIdx, fragments[broadcastIdx]);
} else {
+ LOG.info("[Distributed Join Strategy] : Repartition Join");
// The hash map is modeling as follows:
// <Partition Id, <Table Name, Intermediate Data>>
- Map<Integer, Map<String, List<IntermediateEntry>>> hashEntries =
- new HashMap<Integer, Map<String, List<IntermediateEntry>>>();
+ Map<Integer, Map<String, List<IntermediateEntry>>> hashEntries = new HashMap<Integer, Map<String, List<IntermediateEntry>>>();
// Grouping IntermediateData by a partition key and a table name
for (ScanNode scan : scans) {
- SubQuery childSubQuery = subQuery.getChildQuery(scan);
+ SubQuery childSubQuery = masterContext.getSubQuery(TajoIdUtils.createExecutionBlockId(scan.getTableName()));
for (QueryUnit task : childSubQuery.getQueryUnits()) {
if (task.getIntermediateData() != null) {
for (IntermediateEntry intermEntry : task.getIntermediateData()) {
@@ -112,15 +137,15 @@ public class Repartitioner {
Map<String, List<IntermediateEntry>> tbNameToInterm =
hashEntries.get(intermEntry.getPartitionId());
- if (tbNameToInterm.containsKey(scan.getTableId())) {
- tbNameToInterm.get(scan.getTableId()).add(intermEntry);
+ if (tbNameToInterm.containsKey(scan.getTableName())) {
+ tbNameToInterm.get(scan.getTableName()).add(intermEntry);
} else {
- tbNameToInterm.put(scan.getTableId(), TUtil.newList(intermEntry));
+ tbNameToInterm.put(scan.getTableName(), TUtil.newList(intermEntry));
}
} else {
Map<String, List<IntermediateEntry>> tbNameToInterm =
new HashMap<String, List<IntermediateEntry>>();
- tbNameToInterm.put(scan.getTableId(), TUtil.newList(intermEntry));
+ tbNameToInterm.put(scan.getTableName(), TUtil.newList(intermEntry));
hashEntries.put(intermEntry.getPartitionId(), tbNameToInterm);
}
}
@@ -174,6 +199,41 @@ public class Repartitioner {
return tasks;
}
+ private static QueryUnit [] createLeafTasksWithBroadcastTable(SubQuery subQuery, int baseScanId, Fragment broadcasted) throws IOException {
+ ExecutionBlock execBlock = subQuery.getBlock();
+ ScanNode[] scans = execBlock.getScanNodes();
+ Preconditions.checkArgument(scans.length == 2, "Must be Join Query");
+ TableMeta meta;
+ Path inputPath;
+ ScanNode scan = scans[baseScanId];
+ TableDesc desc = subQuery.getContext().getTableDescMap().get(scan.getTableName());
+ inputPath = desc.getPath();
+ meta = desc.getMeta();
+
+ FileSystem fs = inputPath.getFileSystem(subQuery.getContext().getConf());
+ List<Fragment> fragments = subQuery.getStorageManager().getSplits(scan.getTableName(), meta, inputPath);
+ QueryUnit queryUnit;
+ List<QueryUnit> queryUnits = new ArrayList<QueryUnit>();
+
+ int i = 0;
+ for (Fragment fragment : fragments) {
+ queryUnit = newQueryUnit(subQuery, i++, fragment);
+ queryUnit.setFragment2(broadcasted);
+ queryUnits.add(queryUnit);
+ }
+ return queryUnits.toArray(new QueryUnit[queryUnits.size()]);
+ }
+
+ private static QueryUnit newQueryUnit(SubQuery subQuery, int taskId, Fragment fragment) {
+ ExecutionBlock execBlock = subQuery.getBlock();
+ QueryUnit unit = new QueryUnit(
+ QueryIdFactory.newQueryUnitId(subQuery.getId(), taskId), execBlock.isLeafBlock(),
+ subQuery.getEventHandler());
+ unit.setLogicalPlan(execBlock.getPlan());
+ unit.setFragment2(fragment);
+ return unit;
+ }
+
private static QueryUnit [] newEmptyJoinTask(SubQuery subQuery, Fragment [] fragments, int taskNum) {
ExecutionBlock execBlock = subQuery.getBlock();
QueryUnit [] tasks = new QueryUnit[taskNum];
@@ -193,22 +253,22 @@ public class Repartitioner {
private static void addJoinPartition(QueryUnit task, SubQuery subQuery, int partitionId,
Map<String, List<IntermediateEntry>> grouppedPartitions) {
- for (ScanNode scanNode : subQuery.getBlock().getScanNodes()) {
+ for (ExecutionBlock execBlock : subQuery.getMasterPlan().getChilds(subQuery.getId())) {
Map<String, List<IntermediateEntry>> requests;
- if (grouppedPartitions.containsKey(scanNode.getTableId())) {
- requests = mergeHashPartitionRequest(grouppedPartitions.get(scanNode.getTableId()));
+ if (grouppedPartitions.containsKey(execBlock.getId().toString())) {
+ requests = mergeHashPartitionRequest(grouppedPartitions.get(execBlock.getId().toString()));
} else {
return;
}
Set<URI> fetchURIs = TUtil.newHashSet();
for (Entry<String, List<IntermediateEntry>> requestPerNode : requests.entrySet()) {
Collection<URI> uris = createHashFetchURL(requestPerNode.getKey(),
- subQuery.getChildQuery(scanNode).getId(),
- partitionId, PartitionType.HASH,
+ execBlock.getId(),
+ partitionId, HASH_PARTITION,
requestPerNode.getValue());
fetchURIs.addAll(uris);
}
- task.addFetches(scanNode.getTableId(), fetchURIs);
+ task.addFetches(execBlock.getId().toString(), fetchURIs);
}
}
@@ -233,23 +293,20 @@ public class Repartitioner {
return mergedPartitions;
}
- public static QueryUnit [] createNonLeafTask(SubQuery subQuery,
- SubQuery childSubQuery,
- int maxNum)
+ public static QueryUnit [] createNonLeafTask(MasterPlan masterPlan, SubQuery subQuery, SubQuery childSubQuery,
+ DataChannel channel, int maxNum)
throws InternalException {
- ExecutionBlock childExecBlock = childSubQuery.getBlock();
- if (childExecBlock.getPartitionType() == PartitionType.HASH) {
- return createHashPartitionedTasks(subQuery, childSubQuery, maxNum);
- } else if (childExecBlock.getPartitionType() == PartitionType.RANGE) {
- return createRangePartitionedTasks(subQuery, childSubQuery, maxNum);
+ if (channel.getPartitionType() == HASH_PARTITION) {
+ return createHashPartitionedTasks(masterPlan, subQuery, childSubQuery, channel, maxNum);
+ } else if (channel.getPartitionType() == RANGE_PARTITION) {
+ return createRangePartitionedTasks(subQuery, childSubQuery, channel, maxNum);
} else {
throw new InternalException("Cannot support partition type");
}
}
public static QueryUnit [] createRangePartitionedTasks(SubQuery subQuery,
- SubQuery childSubQuery,
- int maxNum)
+ SubQuery childSubQuery, DataChannel channel, int maxNum)
throws InternalException {
ExecutionBlock execBlock = subQuery.getBlock();
TableStat stat = childSubQuery.getTableStat();
@@ -259,19 +316,15 @@ public class Repartitioner {
ScanNode scan = execBlock.getScanNodes()[0];
Path tablePath;
- tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableId());
+ tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableName());
- StoreTableNode store = (StoreTableNode) childSubQuery.getBlock().getPlan();
- SortNode sort = (SortNode) store.getChild();
- SortSpec[] sortSpecs = sort.getSortKeys();
- Schema sortSchema = PlannerUtil.sortSpecsToSchema(sort.getSortKeys());
+ SortNode sortNode = PlannerUtil.findTopNode(childSubQuery.getBlock().getPlan(), NodeType.SORT);
+ SortSpec [] sortSpecs = sortNode.getSortKeys();
+ Schema sortSchema = new Schema(channel.getPartitionKey());
// calculate the number of maximum query ranges
- TupleRange mergedRange =
- TupleUtil.columnStatToRange(sort.getOutSchema(),
- sortSchema, stat.getColumnStats());
- RangePartitionAlgorithm partitioner =
- new UniformRangePartition(sortSchema, mergedRange);
+ TupleRange mergedRange = TupleUtil.columnStatToRange(channel.getSchema(), sortSchema, stat.getColumnStats());
+ RangePartitionAlgorithm partitioner = new UniformRangePartition(sortSchema, mergedRange);
BigDecimal card = partitioner.getTotalCardinality();
// if the number of the range cardinality is less than the desired number of tasks,
@@ -289,14 +342,12 @@ public class Repartitioner {
" sub ranges (total units: " + determinedTaskNum + ")");
TupleRange [] ranges = partitioner.partition(determinedTaskNum);
- Fragment dummyFragment = new Fragment(scan.getTableId(), tablePath,
+ Fragment dummyFragment = new Fragment(scan.getTableName(), tablePath,
CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV),0, 0);
List<String> basicFetchURIs = new ArrayList<String>();
- SubQuery child = childSubQuery.getContext().getSubQuery(
- subQuery.getBlock().getChildBlock(scan).getId());
- for (QueryUnit qu : child.getQueryUnits()) {
+ for (QueryUnit qu : childSubQuery.getQueryUnits()) {
for (IntermediateEntry p : qu.getIntermediateData()) {
String uri = createBasicFetchUri(p.getPullHost(), p.getPullPort(),
childSubQuery.getId(), p.taskId, p.attemptId);
@@ -330,7 +381,7 @@ public class Repartitioner {
}
QueryUnit [] tasks = createEmptyNonLeafTasks(subQuery, determinedTaskNum, dummyFragment);
- assignPartitionByRoundRobin(map, scan.getTableId(), tasks);
+ assignPartitionByRoundRobin(map, scan.getTableName(), tasks);
return tasks;
}
@@ -367,50 +418,58 @@ public class Repartitioner {
return sb.toString();
}
- public static QueryUnit [] createHashPartitionedTasks(SubQuery subQuery,
- SubQuery childSubQuery,
- int maxNum) {
+ public static QueryUnit [] createHashPartitionedTasks(MasterPlan masterPlan, SubQuery subQuery,
+ SubQuery childSubQuery, DataChannel channel, int maxNum) {
ExecutionBlock execBlock = subQuery.getBlock();
- TableStat stat = childSubQuery.getTableStat();
- if (stat.getNumRows() == 0) {
+
+ List<TableStat> tableStats = new ArrayList<TableStat>();
+ List<ExecutionBlock> childBlocks = masterPlan.getChilds(subQuery.getId());
+ for (ExecutionBlock childBlock : childBlocks) {
+ SubQuery childExecSM = subQuery.getContext().getSubQuery(childBlock.getId());
+ tableStats.add(childExecSM.getTableStat());
+ }
+ TableStat totalStat = StatisticsUtil.computeStatFromUnionBlock(tableStats);
+
+ if (totalStat.getNumRows() == 0) {
return new QueryUnit[0];
}
ScanNode scan = execBlock.getScanNodes()[0];
Path tablePath;
- tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableId());
+ tablePath = subQuery.getContext().getStorageManager().getTablePath(scan.getTableName());
- List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
- for (QueryUnit tasks : childSubQuery.getQueryUnits()) {
- if (tasks.getIntermediateData() != null) {
- partitions.addAll(tasks.getIntermediateData());
- }
- }
- Fragment frag = new Fragment(scan.getTableId(), tablePath,
+ Fragment frag = new Fragment(scan.getTableName(), tablePath,
CatalogUtil.newTableMeta(scan.getInSchema(), StoreType.CSV), 0, 0);
- Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);
+
Map<String, List<IntermediateEntry>> hashedByHost;
Map<Integer, List<URI>> finalFetchURI = new HashMap<Integer, List<URI>>();
- for (Entry<Integer, List<IntermediateEntry>> interm : hashed.entrySet()) {
- hashedByHost = hashByHost(interm.getValue());
- for (Entry<String, List<IntermediateEntry>> e : hashedByHost.entrySet()) {
- Collection<URI> uris = createHashFetchURL(e.getKey(), childSubQuery.getId(),
- interm.getKey(),
- childSubQuery.getBlock().getPartitionType(), e.getValue());
-
- if (finalFetchURI.containsKey(interm.getKey())) {
- finalFetchURI.get(interm.getKey()).addAll(uris);
- } else {
- finalFetchURI.put(interm.getKey(), TUtil.newList(uris));
+ for (ExecutionBlock block : childBlocks) {
+ List<IntermediateEntry> partitions = new ArrayList<IntermediateEntry>();
+ for (QueryUnit tasks : subQuery.getContext().getSubQuery(block.getId()).getQueryUnits()) {
+ if (tasks.getIntermediateData() != null) {
+ partitions.addAll(tasks.getIntermediateData());
+ }
+ }
+ Map<Integer, List<IntermediateEntry>> hashed = hashByKey(partitions);
+ for (Entry<Integer, List<IntermediateEntry>> interm : hashed.entrySet()) {
+ hashedByHost = hashByHost(interm.getValue());
+ for (Entry<String, List<IntermediateEntry>> e : hashedByHost.entrySet()) {
+ Collection<URI> uris = createHashFetchURL(e.getKey(), block.getId(),
+ interm.getKey(), channel.getPartitionType(), e.getValue());
+
+ if (finalFetchURI.containsKey(interm.getKey())) {
+ finalFetchURI.get(interm.getKey()).addAll(uris);
+ } else {
+ finalFetchURI.put(interm.getKey(), TUtil.newList(uris));
+ }
}
}
}
- GroupbyNode groupby = (GroupbyNode) childSubQuery.getBlock().getStoreTableNode().
- getChild();
+ GroupbyNode groupby = (GroupbyNode) childSubQuery.getBlock().getPlan();
// the number of tasks cannot exceed the number of merged fetch uris.
int determinedTaskNum = Math.min(maxNum, finalFetchURI.size());
if (groupby.getGroupingColumns().length == 0) {
@@ -422,7 +481,7 @@ public class Repartitioner {
int tid = 0;
for (Entry<Integer, List<URI>> entry : finalFetchURI.entrySet()) {
for (URI uri : entry.getValue()) {
- tasks[tid].addFetch(scan.getTableId(), uri);
+ tasks[tid].addFetch(scan.getTableName(), uri);
}
tid ++;
@@ -436,8 +495,7 @@ public class Repartitioner {
}
public static Collection<URI> createHashFetchURL(String hostAndPort, ExecutionBlockId ebid,
- int partitionId, PartitionType type,
- List<IntermediateEntry> entries) {
+ int partitionId, PartitionType type, List<IntermediateEntry> entries) {
String scheme = "http://";
StringBuilder urlPrefix = new StringBuilder(scheme);
urlPrefix.append(hostAndPort).append("/?")
@@ -445,9 +503,9 @@ public class Repartitioner {
.append("&sid=").append(ebid.getId())
.append("&p=").append(partitionId)
.append("&type=");
- if (type == PartitionType.HASH) {
+ if (type == HASH_PARTITION) {
urlPrefix.append("h");
- } else if (type == PartitionType.RANGE) {
+ } else if (type == RANGE_PARTITION) {
urlPrefix.append("r");
}
urlPrefix.append("&ta=");
@@ -536,31 +594,35 @@ public class Repartitioner {
return hashed;
}
- public static SubQuery setPartitionNumberForTwoPhase(SubQuery subQuery, final int n) {
+ public static SubQuery setPartitionNumberForTwoPhase(SubQuery subQuery, final int n, DataChannel channel) {
ExecutionBlock execBlock = subQuery.getBlock();
Column[] keys = null;
// if the next query is join,
// set the partition number for the current logicalUnit
// TODO: the union handling is required when a join has unions as its child
+ MasterPlan masterPlan = subQuery.getMasterPlan();
ExecutionBlock parentBlock = execBlock.getParentBlock();
if (parentBlock != null) {
if (parentBlock.getStoreTableNode().getChild().getType() == NodeType.JOIN) {
execBlock.getStoreTableNode().setPartitions(execBlock.getPartitionType(),
execBlock.getStoreTableNode().getPartitionKeys(), n);
keys = execBlock.getStoreTableNode().getPartitionKeys();
+
+ masterPlan.getOutgoingChannels(subQuery.getId()).iterator().next()
+ .setPartition(execBlock.getPartitionType(), execBlock.getStoreTableNode().getPartitionKeys(), n);
}
}
- StoreTableNode store = execBlock.getStoreTableNode();
+
// set the partition number for group by and sort
- if (execBlock.getPartitionType() == PartitionType.HASH) {
- if (store.getChild().getType() == NodeType.GROUP_BY) {
- GroupbyNode groupby = (GroupbyNode)store.getChild();
+ if (channel.getPartitionType() == HASH_PARTITION) {
+ if (execBlock.getPlan().getType() == NodeType.GROUP_BY) {
+ GroupbyNode groupby = (GroupbyNode) execBlock.getPlan();
keys = groupby.getGroupingColumns();
}
- } else if (execBlock.getPartitionType() == PartitionType.RANGE) {
- if (store.getChild().getType() == NodeType.SORT) {
- SortNode sort = (SortNode)store.getChild();
+ } else if (channel.getPartitionType() == RANGE_PARTITION) {
+ if (execBlock.getPlan().getType() == NodeType.SORT) {
+ SortNode sort = (SortNode) execBlock.getPlan();
keys = new Column[sort.getSortKeys().length];
for (int i = 0; i < keys.length; i++) {
keys[i] = sort.getSortKeys()[i].getSortKey();
@@ -569,12 +631,10 @@ public class Repartitioner {
}
if (keys != null) {
if (keys.length == 0) {
- store.setPartitions(execBlock.getPartitionType(), new Column[]{}, 1);
+ channel.setPartition(execBlock.getPartitionType(), new Column[]{}, 1);
} else {
- store.setPartitions(execBlock.getPartitionType(), keys, n);
+ channel.setPartition(execBlock.getPartitionType(), keys, n);
}
- } else {
- store.setListPartition();
}
return subQuery;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 1bf45ee..01d7e46 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -30,23 +30,29 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.state.*;
import org.apache.hadoop.yarn.util.Records;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryIdFactory;
import org.apache.tajo.QueryUnitId;
import org.apache.tajo.catalog.CatalogUtil;
+import org.apache.tajo.catalog.Options;
import org.apache.tajo.catalog.TableDesc;
import org.apache.tajo.catalog.TableMeta;
+import org.apache.tajo.catalog.proto.CatalogProtos;
import org.apache.tajo.catalog.statistics.ColumnStat;
import org.apache.tajo.catalog.statistics.StatisticsUtil;
import org.apache.tajo.catalog.statistics.TableStat;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.planner.logical.GroupbyNode;
import org.apache.tajo.engine.planner.logical.NodeType;
import org.apache.tajo.engine.planner.logical.ScanNode;
-import org.apache.tajo.engine.planner.logical.StoreTableNode;
-import org.apache.tajo.master.*;
+import org.apache.tajo.master.ExecutionBlock;
+import org.apache.tajo.master.TaskRunnerGroupEvent;
import org.apache.tajo.master.TaskRunnerGroupEvent.EventType;
+import org.apache.tajo.master.TaskScheduler;
+import org.apache.tajo.master.TaskSchedulerImpl;
import org.apache.tajo.master.event.*;
import org.apache.tajo.storage.AbstractStorageManager;
import org.apache.tajo.storage.Fragment;
@@ -68,10 +74,11 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
private static final Log LOG = LogFactory.getLog(SubQuery.class);
- private QueryContext queryContext;
+ private MasterPlan masterPlan;
private ExecutionBlock block;
private int priority;
private TableMeta meta;
+ private TableStat statistics;
private EventHandler eventHandler;
private final AbstractStorageManager sm;
private TaskSchedulerImpl taskScheduler;
@@ -135,8 +142,9 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
private int completedTaskCount = 0;
- public SubQuery(QueryMasterTask.QueryMasterTaskContext context, ExecutionBlock block, AbstractStorageManager sm) {
+ public SubQuery(QueryMasterTask.QueryMasterTaskContext context, MasterPlan masterPlan, ExecutionBlock block, AbstractStorageManager sm) {
this.context = context;
+ this.masterPlan = masterPlan;
this.block = block;
this.sm = sm;
this.eventHandler = context.getEventHandler();
@@ -156,6 +164,14 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return context;
}
+ public MasterPlan getMasterPlan() {
+ return masterPlan;
+ }
+
+ public DataChannel getDataChannel() {
+ return masterPlan.getOutgoingChannels(getId()).iterator().next();
+ }
+
public EventHandler getEventHandler() {
return eventHandler;
}
@@ -234,10 +250,6 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return sm;
}
- public SubQuery getChildQuery(ScanNode scanForChild) {
- return context.getSubQuery(block.getChildBlock(scanForChild).getId());
- }
-
public ExecutionBlockId getId() {
return block.getId();
}
@@ -250,17 +262,13 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return tasks.get(qid);
}
- public void setTableMeta(TableMeta meta) {
- this.meta = meta;
- }
-
@SuppressWarnings("UnusedDeclaration")
public TableMeta getTableMeta() {
return meta;
}
public TableStat getTableStat() {
- return this.meta.getStat();
+ return statistics;
}
public String toString() {
@@ -296,17 +304,18 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
}
}
- private static TableStat computeStatFromUnionBlock(SubQuery unit) {
+ public static TableStat computeStatFromUnionBlock(SubQuery subQuery) {
TableStat stat = new TableStat();
TableStat childStat;
long avgRows = 0, numBytes = 0, numRows = 0;
int numBlocks = 0, numPartitions = 0;
List<ColumnStat> columnStats = Lists.newArrayList();
- Iterator<ExecutionBlock> it = unit.getBlock().getChildBlocks().iterator();
+ MasterPlan masterPlan = subQuery.getMasterPlan();
+ Iterator<ExecutionBlock> it = masterPlan.getChilds(subQuery.getBlock()).iterator();
while (it.hasNext()) {
ExecutionBlock block = it.next();
- SubQuery childSubQuery = unit.context.getSubQuery(block.getId());
+ SubQuery childSubQuery = subQuery.context.getSubQuery(block.getId());
childStat = childSubQuery.getTableStat();
avgRows += childStat.getAvgRows();
columnStats.addAll(childStat.getColumnStats());
@@ -325,22 +334,6 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return stat;
}
- public TableMeta buildTableMeta() throws IOException {
- finishTime = context.getClock().getTime();
-
- TableStat stat;
- if (block.hasUnion()) {
- stat = computeStatFromUnionBlock(this);
- } else {
- stat = computeStatFromTasks();
- }
-
- StoreTableNode storeTableNode = getBlock().getStoreTableNode();
- TableMeta meta = toTableMeta(storeTableNode);
- meta.setStat(stat);
- return meta;
- }
-
private TableStat computeStatFromTasks() {
List<TableStat> stats = Lists.newArrayList();
for (QueryUnit unit : getQueryUnits()) {
@@ -350,26 +343,6 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return tableStat;
}
- private TableMeta writeStat(SubQuery subQuery, TableStat stat)
- throws IOException {
- ExecutionBlock execBlock = subQuery.getBlock();
- StoreTableNode storeTableNode = execBlock.getStoreTableNode();
- TableMeta meta = toTableMeta(storeTableNode);
- meta.setStat(stat);
- //sm.writeTableMeta(sm.getTablePath(execBlock.getOutputName()), meta);
- return meta;
- }
-
- private static TableMeta toTableMeta(StoreTableNode store) {
- if (store.hasOptions()) {
- return CatalogUtil.newTableMeta(store.getOutSchema(),
- store.getStorageType(), store.getOptions());
- } else {
- return CatalogUtil.newTableMeta(store.getOutSchema(),
- store.getStorageType());
- }
- }
-
private void stopScheduler() {
// If there are launched TaskRunners, send the 'shouldDie' message to all r
// via received task requests.
@@ -385,15 +358,19 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
}
private void finish() {
- TableMeta meta = null;
- try {
- meta = buildTableMeta();
- } catch (IOException e) {
- e.printStackTrace();
+ TableStat stat;
+ if (block.hasUnion()) {
+ stat = computeStatFromUnionBlock(this);
+ } else {
+ stat = computeStatFromTasks();
}
- setTableMeta(meta);
+ DataChannel channel = masterPlan.getOutgoingChannels(getId()).get(0);
+ meta = CatalogUtil.newTableMeta(channel.getSchema(), CatalogProtos.StoreType.CSV, new Options());
+ meta.setStat(stat);
+ statistics = stat;
setFinishTime();
+
eventHandler.handle(new SubQuerySucceeEvent(getId(), meta));
}
@@ -445,7 +422,8 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
subQuery.finish();
state = SubQueryState.SUCCEEDED;
} else {
- setRepartitionIfNecessary(subQuery);
+ DataChannel channel = subQuery.getMasterPlan().getOutgoingChannels(subQuery.getId()).get(0);
+ setRepartitionIfNecessary(subQuery, channel);
createTasks(subQuery);
if (subQuery.tasks.size() == 0) { // if there is no tasks
@@ -479,10 +457,10 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
* If a parent block requires a repartition operation, the method sets proper repartition
* methods and the number of partitions to a given subquery.
*/
- private static void setRepartitionIfNecessary(SubQuery subQuery) {
+ private static void setRepartitionIfNecessary(SubQuery subQuery, DataChannel channel) {
if (subQuery.getBlock().hasParentBlock()) {
- int numTasks = calculatePartitionNum(subQuery);
- Repartitioner.setPartitionNumberForTwoPhase(subQuery, numTasks);
+ int numTasks = calculatePartitionNum(subQuery, channel);
+ Repartitioner.setPartitionNumberForTwoPhase(subQuery, numTasks, channel);
}
}
@@ -493,14 +471,13 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
* @param subQuery
* @return
*/
- public static int calculatePartitionNum(SubQuery subQuery) {
+ public static int calculatePartitionNum(SubQuery subQuery, DataChannel channel) {
TajoConf conf = subQuery.context.getConf();
ExecutionBlock parent = subQuery.getBlock().getParentBlock();
GroupbyNode grpNode = null;
if (parent != null) {
- grpNode = (GroupbyNode) PlannerUtil.findTopNode(
- parent.getPlan(), NodeType.GROUP_BY);
+ grpNode = PlannerUtil.findTopNode(parent.getPlan(), NodeType.GROUP_BY);
}
// Is this subquery the first step of join?
@@ -509,11 +486,11 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
// for inner
ExecutionBlock outer = child.next();
- long outerVolume = getInputVolume(subQuery.context, outer);
+ long outerVolume = getInputVolume(subQuery.masterPlan, subQuery.context, outer);
// for inner
ExecutionBlock inner = child.next();
- long innerVolume = getInputVolume(subQuery.context, inner);
+ long innerVolume = getInputVolume(subQuery.masterPlan, subQuery.context, inner);
LOG.info("Outer volume: " + Math.ceil((double)outerVolume / 1048576));
LOG.info("Inner volume: " + Math.ceil((double)innerVolume / 1048576));
@@ -533,7 +510,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
if (grpNode.getGroupingColumns().length == 0) {
return 1;
} else {
- long volume = getInputVolume(subQuery.context, subQuery.block);
+ long volume = getInputVolume(subQuery.masterPlan, subQuery.context, subQuery.block);
int mb = (int) Math.ceil((double)volume / 1048576);
LOG.info("Table's volume is approximately " + mb + " MB");
@@ -545,7 +522,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
}
} else {
LOG.info("============>>>>> Unexpected Case! <<<<<================");
- long volume = getInputVolume(subQuery.context, subQuery.block);
+ long volume = getInputVolume(subQuery.masterPlan, subQuery.context, subQuery.block);
int mb = (int) Math.ceil((double)volume / 1048576);
LOG.info("Table's volume is approximately " + mb + " MB");
@@ -557,9 +534,10 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
}
private static void createTasks(SubQuery subQuery) throws IOException {
+ MasterPlan masterPlan = subQuery.getMasterPlan();
ExecutionBlock execBlock = subQuery.getBlock();
QueryUnit [] tasks;
- if (execBlock.isLeafBlock() && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
+ if (subQuery.getMasterPlan().isLeaf(execBlock.getId()) && execBlock.getScanNodes().length == 1) { // Case 1: Just Scan
tasks = createLeafTasks(subQuery);
} else if (execBlock.getScanNodes().length > 1) { // Case 2: Join
@@ -567,9 +545,10 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
} else { // Case 3: Others (Sort or Aggregation)
int numTasks = getNonLeafTaskNum(subQuery);
- ExecutionBlockId childId = subQuery.getBlock().getChildBlocks().iterator().next().getId();
+ ExecutionBlockId childId = masterPlan.getChilds(subQuery.getBlock()).get(0).getId();
SubQuery child = subQuery.context.getSubQuery(childId);
- tasks = Repartitioner.createNonLeafTask(subQuery, child, numTasks);
+ DataChannel channel = masterPlan.getChannel(child.getId(), subQuery.getId());
+ tasks = Repartitioner.createNonLeafTask(masterPlan, subQuery, child, channel, numTasks);
}
LOG.info("Create " + tasks.length + " Tasks");
@@ -587,7 +566,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
*/
public static int getNonLeafTaskNum(SubQuery subQuery) {
// Getting intermediate data size
- long volume = getInputVolume(subQuery.context, subQuery.getBlock());
+ long volume = getInputVolume(subQuery.getMasterPlan(), subQuery.context, subQuery.getBlock());
int mb = (int) Math.ceil((double)volume / 1048576);
LOG.info("Table's volume is approximately " + mb + " MB");
@@ -597,15 +576,15 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
return maxTaskNum;
}
- public static long getInputVolume(QueryMasterTask.QueryMasterTaskContext context, ExecutionBlock execBlock) {
+ public static long getInputVolume(MasterPlan masterPlan, QueryMasterTask.QueryMasterTaskContext context, ExecutionBlock execBlock) {
Map<String, TableDesc> tableMap = context.getTableDescMap();
- if (execBlock.isLeafBlock()) {
+ if (masterPlan.isLeaf(execBlock)) {
ScanNode outerScan = execBlock.getScanNodes()[0];
TableStat stat = tableMap.get(outerScan.getFromTable().getTableName()).getMeta().getStat();
return stat.getNumBytes();
} else {
long aggregatedVolume = 0;
- for (ExecutionBlock childBlock : execBlock.getChildBlocks()) {
+ for (ExecutionBlock childBlock : masterPlan.getChilds(execBlock)) {
SubQuery subquery = context.getSubQuery(childBlock.getId());
aggregatedVolume += subquery.getTableStat().getNumBytes();
}
@@ -650,7 +629,7 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
meta = desc.getMeta();
// TODO - should be change the inner directory
- List<Fragment> fragments = subQuery.getStorageManager().getSplits(scan.getTableId(), meta, inputPath);
+ List<Fragment> fragments = subQuery.getStorageManager().getSplits(scan.getTableName(), meta, inputPath);
QueryUnit queryUnit;
List<QueryUnit> queryUnits = new ArrayList<QueryUnit>();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
index 0286afa..29f44eb 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/TajoWorkerManagerService.java
@@ -202,7 +202,8 @@ public class TajoWorkerManagerService extends CompositeService
QueryId queryId = new QueryId(request.getQueryId());
LOG.info("Receive executeQuery request:" + queryId);
queryMaster.handle(new QueryStartEvent(queryId,
- new QueryContext(request.getQueryContext()), request.getLogicalPlanJson().getValue()));
+ new QueryContext(request.getQueryContext()), request.getSql().getValue(),
+ request.getLogicalPlanJson().getValue()));
done.run(TajoWorker.TRUE_PROTO);
} catch (Exception e) {
LOG.error(e.getMessage(), e);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
index 915234f..63d8f04 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/worker/Task.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.QueryUnitAttemptId;
import org.apache.tajo.TajoConstants;
import org.apache.tajo.TajoProtos.TaskAttemptState;
@@ -48,7 +49,6 @@ import org.apache.tajo.engine.planner.physical.PhysicalExec;
import org.apache.tajo.ipc.TajoWorkerProtocol.*;
import org.apache.tajo.ipc.TajoWorkerProtocol.TajoWorkerProtocolService.Interface;
import org.apache.tajo.ipc.protocolrecords.QueryUnitRequest;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
import org.apache.tajo.master.QueryContext;
import org.apache.tajo.rpc.NullCallback;
import org.apache.tajo.storage.Fragment;
@@ -150,13 +150,15 @@ public class Task {
this.context = new TaskAttemptContext(systemConf, taskId,
request.getFragments().toArray(new Fragment[request.getFragments().size()]),
taskDir);
+ this.context.setDataChannel(request.getDataChannel());
+
plan = CoreGsonHelper.fromJson(request.getSerializedData(), LogicalNode.class);
interQuery = request.getProto().getInterQuery();
if (interQuery) {
context.setInterQuery();
StoreTableNode store = (StoreTableNode) plan;
this.partitionType = store.getPartitionType();
- if (partitionType == PartitionType.RANGE) {
+ if (partitionType == PartitionType.RANGE_PARTITION) {
SortNode sortNode = (SortNode) store.getChild();
this.finalSchema = PlannerUtil.sortSpecsToSchema(sortNode.getSortKeys());
this.sortComp = new TupleComparator(finalSchema, sortNode.getSortKeys());
@@ -180,12 +182,12 @@ public class Task {
LOG.info("* Fragments (num: " + request.getFragments().size() + ")");
for (Fragment f: request.getFragments()) {
- LOG.info("==> Table Id:" + f.getName() + ", path:" + f.getPath() + "(" + f.getMeta().getStoreType() + "), " +
+ LOG.info("Table Id:" + f.getName() + ", path:" + f.getPath() + "(" + f.getMeta().getStoreType() + "), " +
"(start:" + f.getStartOffset() + ", length: " + f.getLength() + ")");
}
LOG.info("* Fetches (total:" + request.getFetches().size() + ") :");
for (Fetch f : request.getFetches()) {
- LOG.info("==> Table Id: " + f.getName() + ", url: " + f.getUrls());
+ LOG.info("Table Id: " + f.getName() + ", url: " + f.getUrls());
}
LOG.info("* Local task dir: " + taskDir);
if(LOG.isDebugEnabled()) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto b/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
index 94bacdf..1726c7b 100644
--- a/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
+++ b/tajo-core/tajo-core-backend/src/main/proto/TajoWorkerProtocol.proto
@@ -60,6 +60,7 @@ message QueryUnitRequestProto {
repeated Fetch fetches = 7;
optional bool shouldDie = 8;
optional KeyValueSetProto queryContext = 9;
+ optional DataChannelProto dataChannel = 10;
}
message Fetch {
@@ -106,7 +107,8 @@ message Partition {
message QueryExecutionRequestProto {
required QueryIdProto queryId = 1;
required KeyValueSetProto queryContext = 2;
- required StringProto logicalPlanJson = 3;
+ required StringProto sql = 3;
+ optional StringProto logicalPlanJson = 4;
}
message GetTaskRequestProto {
@@ -114,6 +116,34 @@ message GetTaskRequestProto {
required ExecutionBlockIdProto executionBlockId = 2;
}
+enum PartitionType {
+ NONE_PARTITION = 0;
+ LIST_PARTITION = 1;
+ HASH_PARTITION = 2;
+ RANGE_PARTITION = 3;
+}
+
+enum TransmitType {
+ PUSH_TRANSMIT = 0;
+ PULL_TRANSMIT = 1;
+ FILE_WRITE = 2;
+}
+
+message DataChannelProto {
+ required ExecutionBlockIdProto srcId = 1;
+ required ExecutionBlockIdProto targetId = 2;
+ required TransmitType transmitType = 3 [default = PULL_TRANSMIT];
+ required PartitionType partitionType = 4;
+
+ optional SchemaProto schema = 5;
+
+ repeated ColumnProto partitionKey = 7;
+ optional int32 partitionNum = 8 [default = 1];
+ repeated SortSpecProto sortSpecs = 9;
+
+ optional StoreType storeType = 10 [default = CSV];
+}
+
message RunExecutionBlockRequestProto {
required string executionBlockId = 1;
required string queryMasterHost = 2;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/resources/webapps/worker/querydetail.jsp
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/resources/webapps/worker/querydetail.jsp b/tajo-core/tajo-core-backend/src/main/resources/webapps/worker/querydetail.jsp
index eaaafe5..c0718a8 100644
--- a/tajo-core/tajo-core-backend/src/main/resources/webapps/worker/querydetail.jsp
+++ b/tajo-core/tajo-core-backend/src/main/resources/webapps/worker/querydetail.jsp
@@ -1,33 +1,45 @@
<%@ page language="java" contentType="text/html; charset=UTF-8" pageEncoding="UTF-8"%>
-<%@ page import="java.util.*" %>
-<%@ page import="java.net.InetSocketAddress" %>
-<%@ page import="java.net.InetAddress" %>
-<%@ page import="org.apache.hadoop.conf.Configuration" %>
-<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
-<%@ page import="org.apache.tajo.worker.*" %>
-<%@ page import="org.apache.tajo.master.*" %>
-<%@ page import="org.apache.tajo.master.rm.*" %>
-<%@ page import="org.apache.tajo.catalog.*" %>
-<%@ page import="java.text.SimpleDateFormat" %>
<%@ page import="org.apache.tajo.QueryId" %>
+<%@ page import="org.apache.tajo.master.querymaster.Query" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryMasterTask" %>
+<%@ page import="org.apache.tajo.master.querymaster.QueryUnit" %>
+<%@ page import="org.apache.tajo.master.querymaster.SubQuery" %>
<%@ page import="org.apache.tajo.util.TajoIdUtils" %>
-<%@ page import="org.apache.tajo.master.querymaster.*" %>
+<%@ page import="org.apache.tajo.webapp.StaticHttpServer" %>
+<%@ page import="org.apache.tajo.worker.TajoWorker" %>
+<%@ page import="java.text.SimpleDateFormat" %>
+<%@ page import="java.util.Collection" %>
<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<html>
<head>
- <link rel="stylesheet" type = "text/css" href = "./style.css" />
+ <link rel="stylesheet" type="text/css" href="./style.css"/>
<meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
<title>Query Detail Info</title>
-<%
+ <%
QueryId queryId = TajoIdUtils.parseQueryId(request.getParameter("queryId"));
TajoWorker tajoWorker = (TajoWorker) StaticHttpServer.getInstance().getAttribute("tajo.info.server.object");
QueryMasterTask queryMasterTask = tajoWorker.getWorkerContext()
.getTajoWorkerManagerService().getQueryMaster().getQueryMasterTask(queryId);
Query query = queryMasterTask.getQuery();
+ %>
+ <h1><% out.write(queryId.toString()); %></h1>
+ <h2>Logical Plan</h2>
+ <pre>
+ <%
+ out.write(query.getPlan().getLogicalPlan().toString());
+ %>
+ </pre>
+ <h2>Distributed Query Plan</h2>
+ <pre>
+ <%
+ out.write(query.getPlan().toString());
+ %>
+ </pre>
+ <%
Collection<SubQuery> subQueries = query.getSubQueries();
SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
@@ -40,7 +52,6 @@
<%
QueryUnit[] queryUnits = eachSubQuery.getQueryUnits();
for(QueryUnit eachQueryUnit: queryUnits) {
- //QueryUnitAttempt queryUnitAttempt = eachQueryUnit.getSuccessfulAttempt();
%>
<tr>
<td><%=eachQueryUnit.getId()%></td>
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
index 2af7fb8..40fc45f 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/LocalTajoTestingUtility.java
@@ -30,6 +30,8 @@ import org.apache.tajo.catalog.TableMeta;
import org.apache.tajo.catalog.proto.CatalogProtos;
import org.apache.tajo.client.TajoClient;
import org.apache.tajo.conf.TajoConf;
+import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.util.TajoIdUtils;
import java.io.IOException;
import java.sql.ResultSet;
@@ -41,6 +43,22 @@ public class LocalTajoTestingUtility {
private TajoConf conf;
private TajoClient client;
+ public static QueryUnitAttemptId newQueryUnitAttemptId() {
+ return QueryIdFactory.newQueryUnitAttemptId(
+ QueryIdFactory.newQueryUnitId(new MasterPlan(newQueryId(), null, null).newExecutionBlockId()), 0);
+ }
+ public static QueryUnitAttemptId newQueryUnitAttemptId(MasterPlan plan) {
+ return QueryIdFactory.newQueryUnitAttemptId(QueryIdFactory.newQueryUnitId(plan.newExecutionBlockId()), 0);
+ }
+
+ /**
+ * for test
+ * @return
+ */
+ public synchronized static QueryId newQueryId() {
+ return QueryIdFactory.newQueryId(TajoIdUtils.MASTER_ID_FORMAT.format(0));
+ }
+
public void setup(String[] names,
String[] tablepaths,
Schema[] schemas,
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
index 7b82952..912400b 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestQueryIdFactory.java
@@ -18,6 +18,7 @@
package org.apache.tajo;
+import org.apache.tajo.engine.planner.global.MasterPlan;
import org.junit.Before;
import org.junit.Test;
@@ -31,23 +32,25 @@ public class TestQueryIdFactory {
@Test
public void testNewQueryId() {
- QueryId qid1 = QueryIdFactory.newQueryId();
- QueryId qid2 = QueryIdFactory.newQueryId();
+ QueryId qid1 = LocalTajoTestingUtility.newQueryId();
+ QueryId qid2 = LocalTajoTestingUtility.newQueryId();
assertTrue(qid1.compareTo(qid2) < 0);
}
@Test
public void testNewSubQueryId() {
- QueryId qid = QueryIdFactory.newQueryId();
- ExecutionBlockId subqid1 = QueryIdFactory.newExecutionBlockId(qid);
- ExecutionBlockId subqid2 = QueryIdFactory.newExecutionBlockId(qid);
+ QueryId qid = LocalTajoTestingUtility.newQueryId();
+ MasterPlan plan = new MasterPlan(qid, null, null);
+ ExecutionBlockId subqid1 = plan.newExecutionBlockId();
+ ExecutionBlockId subqid2 = plan.newExecutionBlockId();
assertTrue(subqid1.compareTo(subqid2) < 0);
}
@Test
public void testNewQueryUnitId() {
- QueryId qid = QueryIdFactory.newQueryId();
- ExecutionBlockId subid = QueryIdFactory.newExecutionBlockId(qid);
+ QueryId qid = LocalTajoTestingUtility.newQueryId();
+ MasterPlan plan = new MasterPlan(qid, null, null);
+ ExecutionBlockId subid = plan.newExecutionBlockId();
QueryUnitId quid1 = QueryIdFactory.newQueryUnitId(subid);
QueryUnitId quid2 = QueryIdFactory.newQueryUnitId(subid);
assertTrue(quid1.compareTo(quid2) < 0);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
index 1997159..1929fb3 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/TestTajoIds.java
@@ -20,6 +20,7 @@ package org.apache.tajo;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.util.TajoIdUtils;
import org.junit.Test;
@@ -121,12 +122,12 @@ public class TestTajoIds {
@Test
public void testConstructFromString() {
-// QueryIdFactory.reset();
- QueryId qid1 = QueryIdFactory.newQueryId();
+ QueryId qid1 = LocalTajoTestingUtility.newQueryId();
QueryId qid2 = TajoIdUtils.parseQueryId(qid1.toString());
assertEquals(qid1, qid2);
- ExecutionBlockId sub1 = QueryIdFactory.newExecutionBlockId(qid1);
+ MasterPlan plan1 = new MasterPlan(qid1, null, null);
+ ExecutionBlockId sub1 = plan1.newExecutionBlockId();
ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
assertEquals(sub1, sub2);
@@ -141,12 +142,12 @@ public class TestTajoIds {
@Test
public void testConstructFromPB() {
-// QueryIdFactory.reset();
- QueryId qid1 = QueryIdFactory.newQueryId();
+ QueryId qid1 = LocalTajoTestingUtility.newQueryId();
QueryId qid2 = new QueryId(qid1.getProto());
assertEquals(qid1, qid2);
- ExecutionBlockId sub1 = QueryIdFactory.newExecutionBlockId(qid1);
+ MasterPlan plan = new MasterPlan(qid1, null, null);
+ ExecutionBlockId sub1 = plan.newExecutionBlockId();
ExecutionBlockId sub2 = TajoIdUtils.createExecutionBlockId(sub1.toString());
assertEquals(sub1, sub2);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
index 75a531f..a6fb17a 100644
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/parser/TestSQLAnalyzer.java
@@ -168,6 +168,12 @@ public class TestSQLAnalyzer {
}
@Test
+ public void testSet4() throws IOException {
+ String sql = FileUtil.readTextFile(new File("src/test/queries/set_4.sql"));
+ System.out.println(parseQuery(sql));
+ }
+
+ @Test
public void testDropTable() throws IOException {
String sql = FileUtil.readTextFile(new File("src/test/queries/drop_table.sql"));
parseQuery(sql);
@@ -240,6 +246,12 @@ public class TestSQLAnalyzer {
}
@Test
+ public void testTableSubQuery2() throws IOException {
+ String sql = FileUtil.readTextFile(new File("src/test/queries/table_subquery2.sql"));
+ parseQuery(sql);
+ }
+
+ @Test
public void testInsertIntoTable() throws IOException {
String sql = FileUtil.readTextFile(new File("src/test/queries/insert_into_select_1.sql"));
parseQuery(sql);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
deleted file mode 100644
index 66060ce..0000000
--- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/plan/global/TestGlobalQueryPlanner.java
+++ /dev/null
@@ -1,391 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.plan.global;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.event.AsyncDispatcher;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.TajoTestingCluster;
-import org.apache.tajo.algebra.Expr;
-import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.FunctionType;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
-import org.apache.tajo.common.TajoDataTypes.Type;
-import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.datum.Datum;
-import org.apache.tajo.datum.DatumFactory;
-import org.apache.tajo.engine.eval.TestEvalTree.TestSum;
-import org.apache.tajo.engine.parser.SQLAnalyzer;
-import org.apache.tajo.engine.planner.LogicalOptimizer;
-import org.apache.tajo.engine.planner.LogicalPlan;
-import org.apache.tajo.engine.planner.LogicalPlanner;
-import org.apache.tajo.engine.planner.PlanningException;
-import org.apache.tajo.engine.planner.global.MasterPlan;
-import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.master.ExecutionBlock;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
-import org.apache.tajo.master.GlobalPlanner;
-import org.apache.tajo.master.TajoMaster;
-import org.apache.tajo.storage.*;
-import org.apache.zookeeper.KeeperException;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import static org.junit.Assert.*;
-
-public class TestGlobalQueryPlanner {
-
- private static TajoTestingCluster util;
- private static TajoConf conf;
- private static CatalogService catalog;
- private static GlobalPlanner planner;
- private static Schema schema;
- private static SQLAnalyzer analyzer;
- private static LogicalPlanner logicalPlanner;
- private static LogicalOptimizer optimizer;
- private static QueryId queryId;
- private static AbstractStorageManager sm;
-
- @BeforeClass
- public static void setup() throws Exception {
- util = new TajoTestingCluster();
- util.startCatalogCluster();
-
- int i, j;
-
- schema = new Schema();
- schema.addColumn("id", Type.INT4);
- schema.addColumn("age", Type.INT4);
- schema.addColumn("name", Type.TEXT);
- schema.addColumn("salary", Type.INT4);
-
- TableMeta meta;
-
- conf = new TajoConf(util.getConfiguration());
- catalog = util.getMiniCatalogCluster().getCatalog();
- for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
- catalog.registerFunction(funcDesc);
- }
-
- sm = StorageManagerFactory.getStorageManager(util.getConfiguration());
- FunctionDesc funcDesc = new FunctionDesc("sumtest", TestSum.class, FunctionType.GENERAL,
- CatalogUtil.newDataTypesWithoutLen(Type.INT4),
- CatalogUtil.newDataTypesWithoutLen(Type.INT4));
- catalog.registerFunction(funcDesc);
- FileSystem fs = sm.getFileSystem();
-
- AsyncDispatcher dispatcher = new AsyncDispatcher();
- dispatcher.init(conf);
- dispatcher.start();
-
- planner = new GlobalPlanner(conf, sm,
- dispatcher.getEventHandler());
- analyzer = new SQLAnalyzer();
- logicalPlanner = new LogicalPlanner(catalog);
- optimizer = new LogicalOptimizer();
-
- int tbNum = 2;
- int tupleNum;
- Appender appender;
- Tuple t = new VTuple(4);
- t.put(new Datum[] {
- DatumFactory.createInt4(1), DatumFactory.createInt4(32),
- DatumFactory.createText("h"), DatumFactory.createInt4(10)});
-
- for (i = 0; i < tbNum; i++) {
- meta = CatalogUtil.newTableMeta((Schema) schema.clone(), StoreType.CSV);
- meta.putOption(CSVFile.DELIMITER, ",");
-
- Path dataRoot = sm.getBaseDir();
- Path tablePath = StorageUtil.concatPath(dataRoot, "table"+i, "file.csv");
- if (fs.exists(tablePath.getParent())) {
- fs.delete(tablePath.getParent(), true);
- }
- fs.mkdirs(tablePath.getParent());
- appender = StorageManagerFactory.getStorageManager(conf).getAppender(meta, tablePath);
- appender.init();
- tupleNum = 100;
- for (j = 0; j < tupleNum; j++) {
- appender.addTuple(t);
- }
- appender.close();
-
- TableDesc desc = CatalogUtil.newTableDesc("table" + i, (TableMeta) meta.clone(), tablePath);
- catalog.addTable(desc);
- }
-
- queryId = QueryIdFactory.newQueryId();
- dispatcher.stop();
- }
-
- @AfterClass
- public static void terminate() throws IOException {
- util.shutdownCatalogCluster();
- }
-
- @Test
- public void testScan() throws IOException, PlanningException {
- Expr context = analyzer.parse(
- "select age, sumtest(salary) from table0");
-
- LogicalPlan plan = logicalPlanner.createPlan(context);
- LogicalNode rootNode = optimizer.optimize(plan);
-
-
- MasterPlan globalPlan = planner.build(queryId,
- (LogicalRootNode) rootNode);
-
- ExecutionBlock unit = globalPlan.getRoot();
- assertFalse(unit.hasChildBlock());
- assertEquals(PartitionType.LIST, unit.getPartitionType());
-
- LogicalNode plan2 = unit.getPlan();
- assertEquals(NodeType.STORE, plan2.getType());
- assertEquals(NodeType.SCAN, ((StoreTableNode)plan2).getChild().getType());
- }
-
- @Test
- public void testGroupby() throws IOException, KeeperException,
- InterruptedException, PlanningException {
- Expr context = analyzer.parse(
- "create table store1 as select age, sumtest(salary) from table0 group by age");
- LogicalPlan plan = logicalPlanner.createPlan(context);
- LogicalNode rootNode = optimizer.optimize(plan);
-
- MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
-
- ExecutionBlock next, prev;
-
- next = globalPlan.getRoot();
- assertTrue(next.hasChildBlock());
- assertEquals(PartitionType.LIST, next.getPartitionType());
- for (ScanNode scan : next.getScanNodes()) {
- assertTrue(scan.isLocal());
- }
- assertFalse(next.getStoreTableNode().isLocal());
- Iterator<ExecutionBlock> it= next.getChildBlocks().iterator();
-
- prev = it.next();
- assertFalse(prev.hasChildBlock());
- assertEquals(PartitionType.HASH, prev.getPartitionType());
- assertTrue(prev.getStoreTableNode().isLocal());
- assertFalse(it.hasNext());
-
- ScanNode []scans = prev.getScanNodes();
- assertEquals(1, scans.length);
- assertEquals("table0", scans[0].getTableId());
- assertFalse(scans[0].isLocal());
-
- scans = next.getScanNodes();
- assertEquals(1, scans.length);
- StoreTableNode store = prev.getStoreTableNode();
- assertEquals(store.getTableName(), scans[0].getTableId());
- assertEquals(store.getOutSchema(), scans[0].getInSchema());
- }
-
- @Test
- public void testSort() throws IOException, PlanningException {
- Expr context = analyzer.parse(
- "create table store1 as select age from table0 order by age");
- LogicalPlan plan = logicalPlanner.createPlan(context);
- LogicalNode rootNode = optimizer.optimize(plan);
-
- MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
-
- ExecutionBlock next, prev;
-
- next = globalPlan.getRoot();
- assertEquals(NodeType.PROJECTION,
- next.getStoreTableNode().getChild().getType());
- assertTrue(next.hasChildBlock());
- assertEquals(PartitionType.LIST, next.getPartitionType());
- Iterator<ExecutionBlock> it= next.getChildBlocks().iterator();
-
- prev = it.next();
- assertEquals(NodeType.SORT,
- prev.getStoreTableNode().getChild().getType());
- assertTrue(prev.hasChildBlock());
- assertEquals(PartitionType.LIST, prev.getPartitionType());
- it= prev.getChildBlocks().iterator();
- next = prev;
-
- prev = it.next();
- assertFalse(prev.hasChildBlock());
- assertEquals(PartitionType.RANGE, prev.getPartitionType());
- assertFalse(it.hasNext());
-
- ScanNode []scans = prev.getScanNodes();
- assertEquals(1, scans.length);
- assertEquals("table0", scans[0].getTableId());
-
- scans = next.getScanNodes();
- assertEquals(1, scans.length);
- StoreTableNode store = prev.getStoreTableNode();
- assertEquals(store.getTableName(), scans[0].getTableId());
- assertEquals(store.getOutSchema(), scans[0].getInSchema());
- }
-
- @Test
- public void testJoin() throws IOException, PlanningException {
- Expr expr = analyzer.parse(
- "select table0.age,table0.salary,table1.salary from table0,table1 " +
- "where table0.salary = table1.salary order by table0.age");
- LogicalPlan plan = logicalPlanner.createPlan(expr);
- LogicalNode rootNode = optimizer.optimize(plan);
-
-
- MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
-
- ExecutionBlock next, prev;
-
- // the second phase of the sort
- next = globalPlan.getRoot();
- assertTrue(next.hasChildBlock());
- assertEquals(PartitionType.LIST, next.getPartitionType());
- assertEquals(NodeType.PROJECTION, next.getStoreTableNode().getChild().getType());
- ScanNode []scans = next.getScanNodes();
- assertEquals(1, scans.length);
- Iterator<ExecutionBlock> it= next.getChildBlocks().iterator();
-
- prev = it.next();
- assertEquals(NodeType.SORT, prev.getStoreTableNode().getChild().getType());
- assertEquals(PartitionType.LIST, prev.getPartitionType());
- scans = prev.getScanNodes();
- assertEquals(1, scans.length);
- it= prev.getChildBlocks().iterator();
-
- // the first phase of the sort
- prev = it.next();
- assertEquals(NodeType.SORT, prev.getStoreTableNode().getChild().getType());
- assertEquals(scans[0].getInSchema(), prev.getOutputSchema());
- assertTrue(prev.hasChildBlock());
- assertEquals(PartitionType.RANGE, prev.getPartitionType());
- assertFalse(it.hasNext());
- scans = prev.getScanNodes();
- assertEquals(1, scans.length);
- next = prev;
- it= next.getChildBlocks().iterator();
-
- // the second phase of the join
- prev = it.next();
- assertEquals(NodeType.JOIN, prev.getStoreTableNode().getChild().getType());
- assertEquals(scans[0].getInSchema(), prev.getOutputSchema());
- assertTrue(prev.hasChildBlock());
- assertEquals(PartitionType.LIST, prev.getPartitionType());
- assertFalse(it.hasNext());
- scans = prev.getScanNodes();
- assertEquals(2, scans.length);
- next = prev;
- it= next.getChildBlocks().iterator();
-
- // the first phase of the join
- prev = it.next();
- assertEquals(NodeType.SCAN, prev.getStoreTableNode().getChild().getType());
- assertFalse(prev.hasChildBlock());
- assertEquals(PartitionType.HASH, prev.getPartitionType());
- assertEquals(1, prev.getScanNodes().length);
-
- prev = it.next();
- assertEquals(NodeType.SCAN, prev.getStoreTableNode().getChild().getType());
- assertFalse(prev.hasChildBlock());
- assertEquals(PartitionType.HASH, prev.getPartitionType());
- assertEquals(1, prev.getScanNodes().length);
- assertFalse(it.hasNext());
- }
-
- @Test
- public void testSelectAfterJoin() throws IOException, PlanningException {
- String query = "select table0.name, table1.salary from table0,table1 where table0.name = table1.name and table1.salary > 10";
- Expr context = analyzer.parse(query);
- LogicalPlan plan = logicalPlanner.createPlan(context);
- LogicalNode rootNode = optimizer.optimize(plan);
-
- MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
-
- ExecutionBlock unit = globalPlan.getRoot();
- StoreTableNode store = unit.getStoreTableNode();
- assertEquals(NodeType.JOIN, store.getChild().getType());
- assertTrue(unit.hasChildBlock());
- ScanNode [] scans = unit.getScanNodes();
- assertEquals(2, scans.length);
- ExecutionBlock prev;
- for (ScanNode scan : scans) {
- prev = unit.getChildBlock(scan);
- store = prev.getStoreTableNode();
- assertEquals(NodeType.SCAN, store.getChild().getType());
- }
- }
-
- //@Test
- public void testCubeby() throws IOException, PlanningException {
- Expr expr = analyzer.parse(
- "select age, sum(salary) from table0 group by cube (age, id)");
- LogicalPlan plan = logicalPlanner.createPlan(expr);
- LogicalNode rootNode = optimizer.optimize(plan);
-
- MasterPlan globalPlan = planner.build(queryId, (LogicalRootNode) rootNode);
-
- ExecutionBlock unit = globalPlan.getRoot();
- StoreTableNode store = unit.getStoreTableNode();
- assertEquals(NodeType.PROJECTION, store.getChild().getType());
-
- ScanNode[] scans = unit.getScanNodes();
- assertEquals(1, scans.length);
-
- unit = unit.getChildBlock(scans[0]);
- store = unit.getStoreTableNode();
- assertEquals(NodeType.UNION, store.getChild().getType());
- UnionNode union = (UnionNode) store.getChild();
- assertEquals(NodeType.SCAN, union.getLeftChild().getType());
- assertEquals(NodeType.UNION, union.getRightChild().getType());
- union = (UnionNode) union.getRightChild();
- assertEquals(NodeType.SCAN, union.getLeftChild().getType());
- assertEquals(NodeType.UNION, union.getRightChild().getType());
- union = (UnionNode) union.getRightChild();
- assertEquals(NodeType.SCAN, union.getLeftChild().getType());
- assertEquals(NodeType.SCAN, union.getRightChild().getType());
- assertTrue(unit.hasChildBlock());
-
- String tableId = "";
- for (ScanNode scan : unit.getScanNodes()) {
- ExecutionBlock prev = unit.getChildBlock(scan);
- store = prev.getStoreTableNode();
- assertEquals(NodeType.GROUP_BY, store.getChild().getType());
- GroupbyNode groupby = (GroupbyNode) store.getChild();
- assertEquals(NodeType.SCAN, groupby.getChild().getType());
- if (tableId.equals("")) {
- tableId = store.getTableName();
- } else {
- assertEquals(tableId, store.getTableName());
- }
- assertEquals(1, prev.getScanNodes().length);
- prev = prev.getChildBlock(prev.getScanNodes()[0]);
- store = prev.getStoreTableNode();
- assertEquals(NodeType.GROUP_BY, store.getChild().getType());
- groupby = (GroupbyNode) store.getChild();
- assertEquals(NodeType.SCAN, groupby.getChild().getType());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestGenericDirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestGenericDirectedGraph.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestGenericDirectedGraph.java
new file mode 100644
index 0000000..f4d4a8d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestGenericDirectedGraph.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner;
+
+import org.apache.tajo.engine.planner.graph.DirectedGraphVisitor;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
+import org.junit.Test;
+
+import java.util.Stack;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestGenericDirectedGraph {
+
+ @Test
+ public final void test() {
+ SimpleDirectedGraph<String, Integer> graph = new SimpleDirectedGraph<String, Integer>();
+
+ // root
+ // / \
+ // (1)/ \ (2)
+ // / \
+ // child1 child2
+ // / \
+ // (3)/ \(4)
+ // / \
+ // child3 child4
+ //
+ String root = "root";
+ String child1 = "child1";
+ String child2 = "child2";
+ String child3 = "child3";
+ String child4 = "child4";
+
+ graph.connect(child1, root, 1);
+ graph.connect(child2, root, 2);
+ graph.connect(child3, child2, 3);
+ graph.connect(child4, child2, 4);
+
+ assertTrue(graph.isRoot(root));
+ assertFalse(graph.isLeaf(root));
+
+ assertEquals(2, graph.getChildCount(root));
+ assertEquals(2, graph.getChildCount(child2));
+
+ graph.accept(root, new Visitor());
+ }
+
+ private class Visitor implements DirectedGraphVisitor<String> {
+ @Override
+ public void visit(Stack<String> stack, String s) {
+ System.out.println("===> " + s);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
new file mode 100644
index 0000000..04ed6fa
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/planner/TestLogicalPlan.java
@@ -0,0 +1,116 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.TajoTestingCluster;
+import org.apache.tajo.benchmark.TPCH;
+import org.apache.tajo.catalog.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
+import org.apache.tajo.catalog.statistics.TableStat;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
+import org.apache.tajo.master.TajoMaster;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.tajo.engine.planner.LogicalPlan.BlockType;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestLogicalPlan {
+ private static TajoTestingCluster util;
+ private static TPCH tpch;
+ private static CatalogService catalog;
+ private static SQLAnalyzer sqlAnalyzer = new SQLAnalyzer();
+ private static LogicalPlanner planner;
+ private static LogicalOptimizer optimizer;
+
+ @BeforeClass
+ public static void setup() throws Exception {
+ util = new TajoTestingCluster();
+ util.startCatalogCluster();
+ catalog = util.getMiniCatalogCluster().getCatalog();
+ for (FunctionDesc funcDesc : TajoMaster.initBuiltinFunctions()) {
+ catalog.registerFunction(funcDesc);
+ }
+
+ // TPC-H Schema for Complex Queries
+ String [] tpchTables = {
+ "part", "supplier", "partsupp", "nation", "region", "lineitem", "customer", "orders"
+ };
+ int [] tableVolumns = {
+ 100, 200, 50, 5, 5, 800, 300, 100
+ };
+ tpch = new TPCH();
+ tpch.loadSchemas();
+ tpch.loadOutSchema();
+
+ for (int i = 0; i < tpchTables.length; i++) {
+ TableMeta m = CatalogUtil.newTableMeta(tpch.getSchema(tpchTables[i]), CatalogProtos.StoreType.CSV);
+ TableStat stat = new TableStat();
+ stat.setNumBytes(tableVolumns[i]);
+ m.setStat(stat);
+ TableDesc d = CatalogUtil.newTableDesc(tpchTables[i], m, new Path("file:///"));
+ catalog.addTable(d);
+ }
+ planner = new LogicalPlanner(catalog);
+ optimizer = new LogicalOptimizer();
+ }
+
+ public static void tearDown() {
+ util.shutdownCatalogCluster();
+ }
+
+ @Test
+ public final void testQueryBlockGraph() {
+ LogicalPlan plan = new LogicalPlan(new LogicalPlanner(catalog));
+ LogicalPlan.QueryBlock root = plan.newAndGetBlock(LogicalPlan.ROOT_BLOCK);
+ LogicalPlan.QueryBlock new1 = plan.newAndGetBlock("@new1");
+ LogicalPlan.QueryBlock new2 = plan.newAndGetBlock("@new2");
+
+ plan.getBlockGraph().connect(new1, root, new LogicalPlan.BlockEdge(new1, root, BlockType.TableSubQuery));
+ plan.getBlockGraph().connect(new2, root, new LogicalPlan.BlockEdge(new2, root, BlockType.TableSubQuery));
+
+ SimpleDirectedGraph<LogicalPlan.QueryBlock, LogicalPlan.BlockEdge> graph = plan.getBlockGraph();
+ assertEquals(2, graph.getChildCount(root));
+
+ assertEquals(root, graph.getParent(new1));
+ assertEquals(root, graph.getParent(new2));
+
+ assertTrue(graph.isRoot(root));
+ assertFalse(graph.isRoot(new1));
+ assertFalse(graph.isRoot(new2));
+
+ assertFalse(graph.isLeaf(root));
+ assertTrue(graph.isLeaf(new1));
+ assertTrue(graph.isLeaf(new2));
+
+ Set<LogicalPlan.QueryBlock> result = new HashSet<LogicalPlan.QueryBlock>();
+ result.add(new1);
+ result.add(new2);
+
+ Set<LogicalPlan.QueryBlock> childs = new HashSet<LogicalPlan.QueryBlock>(graph.getChilds(root));
+ assertEquals(result, childs);
+ }
+}
[7/7] git commit: TAJO-184: Refactor GlobalPlanner and global plan
data structure. (hyunsik)
Posted by hy...@apache.org.
TAJO-184: Refactor GlobalPlanner and global plan data structure. (hyunsik)
Project: http://git-wip-us.apache.org/repos/asf/incubator-tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tajo/commit/1b1d1e8c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tajo/tree/1b1d1e8c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tajo/diff/1b1d1e8c
Branch: refs/heads/master
Commit: 1b1d1e8c1a6b82ccc5c3ce4daeb9e3daa309cde4
Parents: 53df7f4
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Sep 16 20:32:05 2013 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Sep 16 20:32:05 2013 +0900
----------------------------------------------------------------------
CHANGES.txt | 2 +
.../org/apache/tajo/catalog/CatalogUtil.java | 23 +-
.../java/org/apache/tajo/catalog/Column.java | 72 +-
.../java/org/apache/tajo/catalog/Schema.java | 96 +-
.../tajo/catalog/statistics/StatisticsUtil.java | 29 +
.../src/main/proto/CatalogProtos.proto | 9 +-
.../org/apache/tajo/catalog/TestColumn.java | 14 +-
.../org/apache/tajo/catalog/TestSchema.java | 15 +-
.../org/apache/tajo/catalog/CatalogServer.java | 13 +-
.../java/org/apache/tajo/ExecutionBlockId.java | 10 -
.../java/org/apache/tajo/QueryIdFactory.java | 8 -
.../main/java/org/apache/tajo/util/TUtil.java | 35 +-
.../tajo-core-backend/benchmark/tpch/q12.tql | 10 +-
.../tajo-core-backend/benchmark/tpch/q13.tql | 23 +-
.../tajo-core-backend/benchmark/tpch/q14.tql | 10 +-
.../tajo-core-backend/benchmark/tpch/q17.tql | 2 +-
.../tajo-core-backend/benchmark/tpch/q2.tql | 58 +-
.../tajo-core-backend/benchmark/tpch/q4.tql | 26 +-
.../tajo-core-backend/benchmark/tpch/q7.tql | 57 +-
.../tajo-core-backend/benchmark/tpch/q8.tql | 60 +-
.../tajo-core-backend/benchmark/tpch/q9.tql | 50 +-
.../main/java/org/apache/tajo/DataChannel.java | 198 ++++
.../org/apache/tajo/TaskAttemptContext.java | 9 +
.../apache/tajo/engine/eval/EvalTreeUtil.java | 10 +-
.../org/apache/tajo/engine/eval/FieldEval.java | 4 +-
.../apache/tajo/engine/parser/SQLAnalyzer.java | 13 +-
.../tajo/engine/planner/AlgebraVisitor.java | 1 +
.../tajo/engine/planner/BaseAlgebraVisitor.java | 11 +
.../engine/planner/BasicLogicalPlanVisitor.java | 12 +
.../planner/ExplainLogicalPlanVisitor.java | 202 ++++
.../apache/tajo/engine/planner/InsertNode.java | 12 +
.../apache/tajo/engine/planner/LogicalPlan.java | 139 ++-
.../tajo/engine/planner/LogicalPlanVisitor.java | 2 +
.../tajo/engine/planner/LogicalPlanner.java | 48 +-
.../engine/planner/PhysicalPlannerImpl.java | 77 +-
.../apache/tajo/engine/planner/PlanString.java | 99 ++
.../apache/tajo/engine/planner/PlannerUtil.java | 141 +--
.../apache/tajo/engine/planner/Projector.java | 4 +-
.../tajo/engine/planner/QueryBlockGraph.java | 91 --
.../org/apache/tajo/engine/planner/Target.java | 10 +-
.../engine/planner/global/GlobalOptimizer.java | 80 --
.../tajo/engine/planner/global/MasterPlan.java | 217 ++++-
.../engine/planner/graph/DirectedGraph.java | 65 ++
.../planner/graph/DirectedGraphVisitor.java | 25 +
.../planner/graph/SimpleDirectedGraph.java | 219 +++++
.../tajo/engine/planner/logical/BinaryNode.java | 18 +-
.../engine/planner/logical/CreateTableNode.java | 7 +
.../engine/planner/logical/DropTableNode.java | 7 +
.../engine/planner/logical/EvalExprNode.java | 6 +
.../tajo/engine/planner/logical/ExceptNode.java | 11 +
.../engine/planner/logical/GroupbyNode.java | 34 +
.../engine/planner/logical/IndexScanNode.java | 2 -
.../engine/planner/logical/IntersectNode.java | 11 +
.../tajo/engine/planner/logical/JoinNode.java | 13 +-
.../tajo/engine/planner/logical/LimitNode.java | 12 +-
.../engine/planner/logical/LogicalNode.java | 6 +-
.../engine/planner/logical/LogicalRootNode.java | 7 +
.../tajo/engine/planner/logical/NodeType.java | 4 +-
.../engine/planner/logical/ProjectionNode.java | 28 +
.../engine/planner/logical/ReceiveNode.java | 10 +-
.../engine/planner/logical/RelationNode.java | 35 +
.../tajo/engine/planner/logical/ScanNode.java | 60 +-
.../engine/planner/logical/SelectionNode.java | 28 +-
.../tajo/engine/planner/logical/SendNode.java | 8 +-
.../tajo/engine/planner/logical/SortNode.java | 18 +-
.../engine/planner/logical/StoreTableNode.java | 32 +-
.../planner/logical/TableSubQueryNode.java | 123 +++
.../tajo/engine/planner/logical/UnaryNode.java | 7 +-
.../tajo/engine/planner/logical/UnionNode.java | 12 +
.../engine/planner/logical/join/JoinTree.java | 4 +-
.../physical/BasicPhysicalExecutorVisitor.java | 127 +++
.../physical/PhysicalExecutorVisitor.java | 37 +
.../planner/physical/PhysicalPlanUtil.java | 40 +
.../physical/PhysicalPlanningException.java | 31 +
.../engine/planner/physical/SeqScanExec.java | 2 +-
.../planner/rewrite/FilterPushDownRule.java | 3 +-
.../planner/rewrite/ProjectionPushDownRule.java | 137 ++-
.../tajo/engine/query/QueryUnitRequestImpl.java | 33 +-
.../engine/query/ResultSetMetaDataImpl.java | 2 +-
.../ipc/protocolrecords/QueryUnitRequest.java | 2 +
.../org/apache/tajo/master/ExecutionBlock.java | 53 +-
.../tajo/master/ExecutionBlockCursor.java | 24 +-
.../org/apache/tajo/master/GlobalEngine.java | 27 +-
.../org/apache/tajo/master/GlobalPlanner.java | 951 +++++++------------
.../org/apache/tajo/master/QueryContext.java | 8 +
.../apache/tajo/master/TajoAsyncDispatcher.java | 2 -
.../apache/tajo/master/TaskSchedulerImpl.java | 26 +-
.../tajo/master/event/QueryStartEvent.java | 8 +-
.../apache/tajo/master/querymaster/Query.java | 52 +-
.../master/querymaster/QueryInProgress.java | 1 +
.../tajo/master/querymaster/QueryMaster.java | 42 +-
.../master/querymaster/QueryMasterTask.java | 64 +-
.../tajo/master/querymaster/QueryUnit.java | 5 +-
.../tajo/master/querymaster/Repartitioner.java | 260 +++--
.../tajo/master/querymaster/SubQuery.java | 133 ++-
.../tajo/worker/TajoWorkerManagerService.java | 3 +-
.../main/java/org/apache/tajo/worker/Task.java | 10 +-
.../src/main/proto/TajoWorkerProtocol.proto | 32 +-
.../resources/webapps/worker/querydetail.jsp | 39 +-
.../apache/tajo/LocalTajoTestingUtility.java | 18 +
.../org/apache/tajo/TestQueryIdFactory.java | 17 +-
.../test/java/org/apache/tajo/TestTajoIds.java | 13 +-
.../tajo/engine/parser/TestSQLAnalyzer.java | 12 +
.../plan/global/TestGlobalQueryPlanner.java | 391 --------
.../planner/TestGenericDirectedGraph.java | 73 ++
.../tajo/engine/planner/TestLogicalPlan.java | 116 +++
.../tajo/engine/planner/TestLogicalPlanner.java | 268 +++---
.../tajo/engine/planner/TestPlannerUtil.java | 25 +-
.../global/TestGlobalQueryOptimizer.java | 171 ----
.../engine/planner/global/TestMasterPlan.java | 54 ++
.../planner/physical/TestBNLJoinExec.java | 5 +-
.../planner/physical/TestBSTIndexExec.java | 12 +-
.../planner/physical/TestExternalSortExec.java | 4 +-
.../planner/physical/TestHashAntiJoinExec.java | 3 +-
.../planner/physical/TestHashJoinExec.java | 3 +-
.../planner/physical/TestHashSemiJoinExec.java | 3 +-
.../planner/physical/TestMergeJoinExec.java | 3 +-
.../engine/planner/physical/TestNLJoinExec.java | 10 +-
.../planner/physical/TestPhysicalPlanner.java | 86 +-
.../engine/planner/physical/TestSortExec.java | 4 +-
.../tajo/engine/query/TestGroupByQuery.java | 37 +-
.../tajo/engine/query/TestResultSetImpl.java | 2 +-
.../tajo/engine/query/TestSelectQuery.java | 5 +-
.../tajo/engine/query/TestTableSubQuery.java | 73 ++
.../tajo/master/TestExecutionBlockCursor.java | 14 +-
.../apache/tajo/master/TestGlobalPlanner.java | 185 ++++
.../apache/tajo/master/TestRepartitioner.java | 4 +-
.../tajo/worker/TestRangeRetrieverHandler.java | 6 +-
.../worker/dataserver/TestHttpDataServer.java | 10 +-
.../src/test/queries/complex_union_1.sql | 29 +
.../src/test/queries/complex_union_2.sql | 35 +
.../src/test/queries/select_13.hiveql | 2 +-
.../src/test/queries/set_4.sql | 36 +
.../src/test/queries/table_subquery2.sql | 21 +
.../src/test/resources/catalog-default.xml | 2 +-
.../tajo/storage/TestTupleComparator.java | 4 +-
.../apache/tajo/storage/index/TestBSTIndex.java | 60 +-
.../index/TestSingleCSVFileBSTIndex.java | 8 +-
138 files changed, 4264 insertions(+), 2528 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 73c3b61..58b90a9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -25,6 +25,8 @@ Release 0.2.0 - unreleased
IMPROVEMENTS
+ TAJO-184: Refactor GlobalPlanner and global plan data structure. (hyunsik)
+
TAJO-119: Refactor and Improve Datum. (jinho)
TAJO-178: Implements StorageManager for scanning asynchronously.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
index 8f17852..fb55bf7 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java
@@ -141,27 +141,14 @@ public class CatalogUtil {
*
* @return
*/
- public static SchemaProto getQualfiedSchema(String tableName,
- SchemaProto schema) {
+ public static SchemaProto getQualfiedSchema(String tableName, SchemaProto schema) {
SchemaProto.Builder revisedSchema = SchemaProto.newBuilder(schema);
revisedSchema.clearFields();
- String[] split;
for (ColumnProto col : schema.getFieldsList()) {
- split = col.getColumnName().split("\\.");
- if (split.length == 1) { // if not qualified name
- // rewrite the column
- ColumnProto.Builder builder = ColumnProto.newBuilder(col);
- builder.setColumnName(tableName + "." + col.getColumnName());
- col = builder.build();
- } else if (split.length == 2) {
- ColumnProto.Builder builder = ColumnProto.newBuilder(col);
- builder.setColumnName(tableName + "." + split[1]);
- col = builder.build();
- } else {
- throw new InternalError("Unaccetable field name "
- + col.getColumnName());
- }
- revisedSchema.addFields(col);
+ ColumnProto.Builder builder = ColumnProto.newBuilder(col);
+ builder.setColumnName(col.getColumnName());
+ builder.setQualifier(tableName);
+ revisedSchema.addFields(builder.build());
}
return revisedSchema.build();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
index 2227999..1b623fa 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Column.java
@@ -18,17 +18,20 @@
package org.apache.tajo.catalog;
+import com.google.common.base.Objects;
import com.google.gson.annotations.Expose;
-import org.apache.tajo.json.GsonObject;
import org.apache.tajo.catalog.json.CatalogGsonHelper;
import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
import org.apache.tajo.common.ProtoObject;
import org.apache.tajo.common.TajoDataTypes;
import org.apache.tajo.common.TajoDataTypes.DataType;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.util.TUtil;
public class Column implements ProtoObject<ColumnProto>, Cloneable, GsonObject {
private ColumnProto.Builder builder = null;
-
+
+ @Expose protected String qualifier; // optional
@Expose protected String name; // required
@Expose protected DataType dataType; // required
@@ -38,7 +41,7 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable, GsonObject {
public Column(String columnName, DataType dataType) {
this();
- this.name = columnName.toLowerCase();
+ checkAndSetName(columnName.toLowerCase());
this.dataType = dataType;
}
@@ -51,34 +54,55 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable, GsonObject {
}
public Column(ColumnProto proto) {
- this(proto.getColumnName(), proto.getDataType());
+ this();
+ name = proto.getColumnName();
+ dataType = proto.getDataType();
+ if (proto.hasQualifier()) {
+ qualifier = proto.getQualifier();
+ }
}
+ private void checkAndSetName(String qualifiedOrName) {
+ String [] splits = qualifiedOrName.split("\\.");
+ if (splits.length > 1) {
+ qualifier = qualifiedOrName.substring(0, qualifiedOrName.lastIndexOf("."));
+ name = qualifiedOrName.substring(qualifiedOrName.lastIndexOf(".") + 1, qualifiedOrName.length());
+ } else {
+ qualifier = null;
+ name = qualifiedOrName;
+ }
+ }
+
public String getQualifiedName() {
- return this.name;
+ if (qualifier != null) {
+ return qualifier + "." + name;
+ } else {
+ return name;
+ }
}
- public boolean isQualified() {
- return getQualifiedName().split("\\.").length == 2;
+ public boolean hasQualifier() {
+ return qualifier != null;
}
- public String getTableName() {
- if (isQualified()) {
- return getQualifiedName().split("\\.")[0];
+ public void setQualifier(String qualifier) {
+ this.qualifier = qualifier;
+ }
+
+ public String getQualifier() {
+ if (qualifier != null) {
+ return qualifier;
} else {
return "";
}
}
public String getColumnName() {
- if (isQualified())
- return getQualifiedName().split("\\.")[1];
- else
- return getQualifiedName();
+ return name;
}
public void setName(String name) {
- this.name = name.toLowerCase();
+ checkAndSetName(name.toLowerCase());
}
public DataType getDataType() {
@@ -92,15 +116,17 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable, GsonObject {
@Override
public boolean equals(Object o) {
if (o instanceof Column) {
- Column cd = (Column)o;
- return this.getQualifiedName().equals(cd.getQualifiedName()) &&
- this.getDataType().equals(cd.getDataType());
- }
+ Column another = (Column)o;
+ return name.equals(another.name) &&
+ dataType.equals(another.dataType) &&
+ TUtil.checkEquals(qualifier, another.qualifier);
+ }
return false;
}
public int hashCode() {
- return getQualifiedName().hashCode() ^ (getDataType().hashCode() * 17);
+ return Objects.hashCode(name, dataType, qualifier);
+
}
@Override
@@ -109,6 +135,7 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable, GsonObject {
column.builder = ColumnProto.newBuilder();
column.name = name;
column.dataType = dataType;
+ column.qualifier = qualifier != null ? qualifier : null;
return column;
}
@@ -116,12 +143,15 @@ public class Column implements ProtoObject<ColumnProto>, Cloneable, GsonObject {
public ColumnProto getProto() {
builder.setColumnName(this.name);
builder.setDataType(this.dataType);
+ if (qualifier != null) {
+ builder.setQualifier(qualifier);
+ }
return builder.build();
}
public String toString() {
- return getQualifiedName() +" (" + getDataType().getType() +")";
+ return getQualifiedName() +" (" + getDataType().getType() +"(" + getDataType().getLength() + "))";
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
index 65f6176..4659a02 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/Schema.java
@@ -21,7 +21,6 @@ package org.apache.tajo.catalog;
import com.google.gson.annotations.Expose;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.tajo.json.GsonObject;
import org.apache.tajo.catalog.exception.AlreadyExistsFieldException;
import org.apache.tajo.catalog.json.CatalogGsonHelper;
import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto;
@@ -29,6 +28,8 @@ import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto;
import org.apache.tajo.common.ProtoObject;
import org.apache.tajo.common.TajoDataTypes.DataType;
import org.apache.tajo.common.TajoDataTypes.Type;
+import org.apache.tajo.json.GsonObject;
+import org.apache.tajo.util.TUtil;
import java.util.*;
@@ -37,26 +38,39 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
private SchemaProto.Builder builder = SchemaProto.newBuilder();
@Expose protected List<Column> fields = null;
- @Expose protected Map<String,Integer> fieldsByName = null;
+ @Expose protected Map<String, Integer> fieldsByQialifiedName = null;
+ @Expose protected Map<String, List<Integer>> fieldsByName = null;
public Schema() {
this.fields = new ArrayList<Column>();
- this.fieldsByName = new TreeMap<String, Integer>();
+ this.fieldsByQialifiedName = new TreeMap<String, Integer>();
+ this.fieldsByName = new HashMap<String, List<Integer>>();
}
public Schema(SchemaProto proto) {
this.fields = new ArrayList<Column>();
- this.fieldsByName = new HashMap<String, Integer>();
+ this.fieldsByQialifiedName = new HashMap<String, Integer>();
+ this.fieldsByName = new HashMap<String, List<Integer>>();
for(ColumnProto colProto : proto.getFieldsList()) {
fields.add(new Column(colProto));
- fieldsByName.put(colProto.getColumnName(), fields.size() - 1);
+ if (colProto.hasQualifier()) {
+ fieldsByQialifiedName.put(colProto.getQualifier() + "." + colProto.getColumnName(), fields.size() - 1);
+ } else {
+ fieldsByQialifiedName.put(colProto.getColumnName(), fields.size() - 1);
+ }
+ if (fieldsByName.containsKey(colProto.getColumnName())) {
+ fieldsByName.get(colProto.getColumnName()).add(fields.size() - 1);
+ } else {
+ fieldsByName.put(colProto.getColumnName(), TUtil.newList(fields.size() - 1));
+ }
}
}
public Schema(Schema schema) {
this();
this.fields.addAll(schema.fields);
- this.fieldsByName.putAll(schema.fieldsByName);
+ this.fieldsByQialifiedName.putAll(schema.fieldsByQialifiedName);
+ this.fieldsByName.putAll(schema.fieldsByName);
}
public Schema(Column [] columns) {
@@ -65,23 +79,48 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
addColumn(c);
}
}
+
+ public void setQualifier(String qualifier) {
+ fieldsByQialifiedName.clear();
+
+ for (int i = 0; i < getColumnNum(); i++) {
+ fields.get(i).setQualifier(qualifier);
+ fieldsByQialifiedName.put(fields.get(i).getQualifiedName(), i);
+ }
+ }
public int getColumnNum() {
return this.fields.size();
}
- public Column getColumn(String colName) {
- Integer cid = fieldsByName.get(colName.toLowerCase());
+ public Column getColumnByFQN(String colName) {
+ Integer cid = fieldsByQialifiedName.get(colName.toLowerCase());
return cid != null ? fields.get(cid) : null;
}
public Column getColumnByName(String colName) {
- for (Column col : fields) {
- if (col.getColumnName().equals(colName.toLowerCase())) {
- return col;
- }
- }
- return null;
+ String normalized = colName.toLowerCase();
+ List<Integer> list = fieldsByName.get(normalized);
+
+ if (list == null || list.size() == 0) {
+ return null;
+ }
+
+ if (list.size() == 1) {
+ return fields.get(list.get(0));
+ } else {
+ StringBuilder sb = new StringBuilder();
+ boolean first = true;
+ for (Integer id : list) {
+ if (first) {
+ first = false;
+ } else {
+ sb.append(", ");
+ }
+ sb.append(fields.get(id));
+ }
+ throw new RuntimeException("Ambiguous Column Name: " + sb.toString());
+ }
}
public Column getColumn(int id) {
@@ -89,13 +128,13 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
}
public int getColumnId(String colName) {
- return fieldsByName.get(colName.toLowerCase());
+ return fieldsByQialifiedName.get(colName.toLowerCase());
}
public int getColumnIdByName(String colName) {
for (Column col : fields) {
if (col.getColumnName().equals(colName.toLowerCase())) {
- return fieldsByName.get(col.getQualifiedName());
+ return fieldsByQialifiedName.get(col.getQualifiedName());
}
}
return -1;
@@ -110,7 +149,7 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
}
public boolean contains(String colName) {
- return fieldsByName.containsKey(colName.toLowerCase());
+ return fieldsByQialifiedName.containsKey(colName.toLowerCase());
}
public synchronized Schema addColumn(String name, Type type) {
@@ -125,21 +164,22 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
}
public synchronized Schema addColumn(String name, DataType dataType) {
- String lowcased = name.toLowerCase();
- if(fieldsByName.containsKey(lowcased)) {
- LOG.error("Already exists column " + lowcased);
- throw new AlreadyExistsFieldException(lowcased);
+ String nomalized = name.toLowerCase();
+ if(fieldsByQialifiedName.containsKey(nomalized)) {
+ LOG.error("Already exists column " + nomalized);
+ throw new AlreadyExistsFieldException(nomalized);
}
- Column newCol = new Column(lowcased, dataType);
+ Column newCol = new Column(nomalized, dataType);
fields.add(newCol);
- fieldsByName.put(lowcased, fields.size() - 1);
+ fieldsByQialifiedName.put(newCol.getQualifiedName(), fields.size() - 1);
+ fieldsByName.put(newCol.getColumnName(), TUtil.newList(fields.size() - 1));
return this;
}
public synchronized void addColumn(Column column) {
- addColumn(column.getQualifiedName(), column.getDataType());
+ addColumn(column.getQualifiedName(), column.getDataType());
}
public synchronized void addColumns(Schema schema) {
@@ -158,12 +198,8 @@ public class Schema implements ProtoObject<SchemaProto>, Cloneable, GsonObject {
}
@Override
- public Object clone() throws CloneNotSupportedException {
- Schema schema = (Schema) super.clone();
- schema.builder = SchemaProto.newBuilder();
- schema.fields = fields != null ? new ArrayList<Column>(fields) : null;
- schema.fieldsByName = fieldsByName != null ? new TreeMap<String, Integer>(fieldsByName) : null;
-
+ public Object clone() {
+ Schema schema = new Schema(toArray());
return schema;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
index 188fdcc..470e1a5 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/statistics/StatisticsUtil.java
@@ -22,6 +22,8 @@ import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import java.util.Collection;
+import java.util.Iterator;
import java.util.List;
public class StatisticsUtil {
@@ -101,4 +103,31 @@ public class StatisticsUtil {
return aggregated;
}
+
+ public static TableStat computeStatFromUnionBlock(Collection<TableStat> stats) {
+ TableStat stat = new TableStat();
+ TableStat childStat;
+ long avgRows = 0, numBytes = 0, numRows = 0;
+ int numBlocks = 0, numPartitions = 0;
+ List<ColumnStat> columnStats = Lists.newArrayList();
+
+ Iterator<TableStat> it = stats.iterator();
+ while (it.hasNext()) {
+ childStat = it.next();
+ avgRows += childStat.getAvgRows();
+ columnStats.addAll(childStat.getColumnStats());
+ numBlocks += childStat.getNumBlocks();
+ numBytes += childStat.getNumBytes();
+ numPartitions += childStat.getNumPartitions();
+ numRows += childStat.getNumRows();
+ }
+
+ stat.setColumnStats(columnStats);
+ stat.setNumBlocks(numBlocks);
+ stat.setNumBytes(numBytes);
+ stat.setNumPartitions(numPartitions);
+ stat.setNumRows(numRows);
+ stat.setAvgRows(avgRows);
+ return stat;
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
index 3d61ebb..477362c 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
+++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto
@@ -64,7 +64,8 @@ message ColumnMetaProto {
message ColumnProto {
required string columnName = 1;
- required DataType dataType = 2;
+ optional string qualifier = 2;
+ required DataType dataType = 3;
}
message SchemaProto {
@@ -130,9 +131,9 @@ message IndexDescProto {
enum IndexMethod {
TWO_LEVEL_BIN_TREE = 0;
- BTREE = 1;
- HASH = 2;
- BITMAP = 3;
+ BTREE_IDX = 1;
+ HASH_IDX = 2;
+ BITMAP_IDX = 3;
}
message GetAllTableNamesResponse {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
index dfea828..406149e 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestColumn.java
@@ -77,12 +77,22 @@ public class TestColumn {
public final void testQualifiedName() {
Column col = new Column("table_1.id", Type.INT4);
- assertTrue(col.isQualified());
+ assertTrue(col.hasQualifier());
assertEquals("id", col.getColumnName());
assertEquals("table_1.id", col.getQualifiedName());
- assertEquals("table_1", col.getTableName());
+ assertEquals("table_1", col.getQualifier());
}
+ @Test
+ public final void testMultiLevelQualifiedName() {
+ Column col = new Column("database1.table_1.id", Type.INT4);
+
+ assertTrue(col.hasQualifier());
+ assertEquals("id", col.getColumnName());
+ assertEquals("database1.table_1.id", col.getQualifiedName());
+ assertEquals("database1.table_1", col.getQualifier());
+ }
+
@Test
public final void testToJson() {
Column col = new Column(field1.getProto());
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
index 689f004..e1be420 100644
--- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
+++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestSchema.java
@@ -61,9 +61,9 @@ public class TestSchema {
@Test
public final void testGetColumnString() {
- assertEquals(col1, schema.getColumn("name"));
- assertEquals(col2, schema.getColumn("age"));
- assertEquals(col3, schema.getColumn("addr"));
+ assertEquals(col1, schema.getColumnByFQN("name"));
+ assertEquals(col2, schema.getColumnByFQN("age"));
+ assertEquals(col3, schema.getColumnByFQN("addr"));
}
@Test
@@ -134,4 +134,13 @@ public class TestSchema {
Schema fromJson = new Schema(proto);
assertEquals(schema2, fromJson);
}
+
+ @Test
+ public final void testSetQualifier() {
+ Schema schema2 = new Schema(schema.getProto());
+ schema2.setQualifier("test1");
+ Column column = schema2.getColumn(1);
+ assertEquals(column, schema2.getColumnByName("age"));
+ assertEquals(column, schema2.getColumnByFQN("test1.age"));
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
index 3ac4ed0..94eae97 100644
--- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
+++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/CatalogServer.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.tajo.catalog.CatalogProtocol.CatalogProtocolService;
import org.apache.tajo.catalog.exception.*;
+import org.apache.tajo.catalog.proto.CatalogProtos;
import org.apache.tajo.catalog.proto.CatalogProtos.*;
import org.apache.tajo.catalog.store.CatalogStore;
import org.apache.tajo.catalog.store.DBStore;
@@ -173,7 +174,11 @@ public class CatalogServer extends AbstractService {
if (!store.existTable(tableId)) {
throw new NoSuchTableException(tableId);
}
- return (TableDescProto) store.getTable(tableId).getProto();
+ TableDesc desc = store.getTable(tableId);
+ SchemaProto schemaProto = desc.getMeta().getSchema().getProto();
+ SchemaProto qualifiedSchema = CatalogUtil.getQualfiedSchema(tableId, schemaProto);
+ desc.getMeta().setSchema(new Schema(qualifiedSchema));
+ return (TableDescProto) desc.getProto();
} catch (IOException ioe) {
// TODO - handle exception
LOG.error(ioe);
@@ -228,12 +233,8 @@ public class CatalogServer extends AbstractService {
}
// rewrite schema
- SchemaProto revisedSchema =
- CatalogUtil.getQualfiedSchema(tableDesc.getId(), tableDesc.getMeta()
- .getSchema());
-
TableProto.Builder metaBuilder = TableProto.newBuilder(tableDesc.getMeta());
- metaBuilder.setSchema(revisedSchema);
+ metaBuilder.setSchema(tableDesc.getMeta().getSchema());
TableDescProto.Builder descBuilder = TableDescProto.newBuilder(tableDesc);
descBuilder.setMeta(metaBuilder.build());
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java b/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
index 2dc4441..1ccb357 100644
--- a/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
+++ b/tajo-common/src/main/java/org/apache/tajo/ExecutionBlockId.java
@@ -32,16 +32,6 @@ public class ExecutionBlockId implements Comparable<ExecutionBlockId> {
this(new QueryId(proto.getQueryId()), proto.getId());
}
-// public ExecutionBlockId(String idStr) {
-// String[] tokens = idStr.split(QueryId.SEPARATOR);
-// if(tokens.length < 3) {
-// throw new RuntimeException("Wrong ExecutionBlockId format[" + idStr + "]");
-// }
-//
-// this.queryId = new QueryId(tokens);
-// this.id = Integer.parseInt(tokens[3]);
-// }
-
@Override
public String toString() {
return EB_ID_PREFIX + QueryId.SEPARATOR + toStringNoPrefix();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java b/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
index 90533e3..b0affe9 100644
--- a/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
+++ b/tajo-common/src/main/java/org/apache/tajo/QueryIdFactory.java
@@ -71,14 +71,6 @@ public class QueryIdFactory {
return new QueryId(String.valueOf(timestamp), seq);
}
- /**
- * for test
- * @return
- */
- public synchronized static QueryId newQueryId() {
- return newQueryId(TajoIdUtils.MASTER_ID_FORMAT.format(0));
- }
-
public synchronized static QueryId newQueryId(String seedQueryId, int seq) {
if(isYarnId(seedQueryId)) {
String[] tokens = seedQueryId.split("_");
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
index 05c0972..373292b 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TUtil.java
@@ -18,9 +18,6 @@
package org.apache.tajo.util;
-import org.apache.tajo.QueryIdFactory;
-import org.apache.tajo.QueryUnitAttemptId;
-
import java.util.*;
/**
@@ -86,6 +83,22 @@ public class TUtil {
return new HashMap<K, V>(map);
}
+ public static <K, V> Map<K,V> newHashMap(K k, V v) {
+ HashMap<K, V> newMap = new HashMap<K, V>();
+ newMap.put(k, v);
+ return newMap;
+ }
+
+ public static <K,V> Map<K,V> newLinkedHashMap() {
+ return new LinkedHashMap<K, V>();
+ }
+
+ public static <K, V> Map<K,V> newLinkedHashMap(K k, V v) {
+ HashMap<K, V> newMap = new LinkedHashMap<K, V>();
+ newMap.put(k, v);
+ return newMap;
+ }
+
public static <T> List<T> newList() {
return new ArrayList<T>();
}
@@ -108,13 +121,6 @@ public class TUtil {
return list;
}
- public static QueryUnitAttemptId newQueryUnitAttemptId() {
- return QueryIdFactory.newQueryUnitAttemptId(
- QueryIdFactory.newQueryUnitId(
- QueryIdFactory.newExecutionBlockId(
- QueryIdFactory.newQueryId())), 0);
- }
-
/**
* It check if T is null or not.
*
@@ -128,4 +134,13 @@ public class TUtil {
}
return reference;
}
+
+ public static <KEY1, KEY2, VALUE> void putToNestedMap(Map<KEY1, Map<KEY2, VALUE>> map, KEY1 k1, KEY2 k2,
+ VALUE value) {
+ if (map.containsKey(k1)) {
+ map.get(k1).put(k2, value);
+ } else {
+ map.put(k1, TUtil.newLinkedHashMap(k2, value));
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q12.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q12.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q12.tql
index cfdd0f5..2f26ee9 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q12.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q12.tql
@@ -1,7 +1,13 @@
select
l_shipmode,
- sum(case when o_orderpriority ='1-URGENT' or o_orderpriority ='2-HIGH' then 1 else 0 end) as high_line_count,
- sum(case when o_orderpriority != '1-URGENT' and o_orderpriority != '2-HIGH' then 1 else 0 end) as low_line_count
+ sum(case when o_orderpriority ='1-URGENT' or o_orderpriority ='2-HIGH'
+ then 1
+ else 0
+ end) as high_line_count,
+ sum(case when o_orderpriority != '1-URGENT' and o_orderpriority != '2-HIGH'
+ then 1
+ else 0
+ end) as low_line_count
from
orders,
lineitem
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q13.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q13.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q13.tql
index a840a23..2ee0102 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q13.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q13.tql
@@ -1,9 +1,18 @@
select
- c_custkey,
- count(o_orderkey)
-from
- customer
-left outer join
- orders on c_custkey = o_custkey and o_comment not like '%special%requests%'
+ c_count, count(*) as custdist
+from (
+ select
+ c_custkey,
+ count(o_orderkey)
+ from
+ customer left outer join orders on
+ c_custkey = o_custkey
+ and o_comment not like '%special%requests%'
+ group by
+ c_custkey
+ ) as c_orders (c_custkey, c_count)
group by
- c_custkey
\ No newline at end of file
+ c_count
+order by
+ custdist desc,
+ c_count desc
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q14.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q14.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q14.tql
index 7614904..f00e469 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q14.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q14.tql
@@ -1,9 +1,13 @@
select
- 100.00 * sum(case when p_type like 'PROMO%' then l_extendedprice*(1-l_discount) else 0 end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue
+ 100.00 * sum(case
+ when p_type like 'PROMO%'
+ then l_extendedprice*(1-l_discount)
+ else 0
+ end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue
from
lineitem,
part
where
l_partkey = p_partkey and
- l_shipdate >= date '1995-09-01' and
- l_shipdate < '1995-10-01'
\ No newline at end of file
+ l_shipdate >= '1995-09-01' and
+ l_shipdate < '1995-10-01';
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q17.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q17.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q17.tql
index bfc7ff6..d67a399 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q17.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q17.tql
@@ -14,4 +14,4 @@ where
lineitem
where
l_partkey = p_partkey
- )
+ )
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q2.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q2.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q2.tql
index c0bf3c2..b7a7cd8 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q2.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q2.tql
@@ -1,24 +1,38 @@
select
- s_acctbal,
- s_name,
- n_name,
- p_partkey,
- p_mfgr,
- s_address,
- s_phone,
- s_comment,
- ps_supplycost
-from
- region
-join
- nation on n_regionkey = r_regionkey
-join
- supplier on s_nationkey = n_nationkey
-join
- partsupp on s_suppkey = ps_suppkey
-join
- part on p_partkey = ps_partkey
+ s_acctbal,
+ s_name,
+ n_name,
+ p_partkey,
+ p_mfgr,
+ s_address,
+ s_phone,
+ s_comment
+from
+ part,
+ supplier,
+ partsupp,
+ nation,
+ region
where
- p_size = 15 and p_type
-like
- '%BRASS' and r_name = 'EUROPE'
\ No newline at end of file
+ p_partkey = ps_partkey
+ and s_suppkey = ps_suppkey
+ and p_size = 15
+ and p_type like '%BRASS'
+ and s_nationkey = n_nationkey
+ and n_regionkey = r_regionkey
+ and r_name = 'c'
+ and ps_supplycost =
+ (
+ select min(ps_supplycost) from partsupp, supplier, nation, region
+ where
+ p_partkey = ps_partkey
+ and s_suppkey = ps_suppkey
+ and s_nationkey = n_nationkey
+ and n_regionkey = r_regionkey
+ and r_name = 'EUROPE'
+ )
+order by
+ s_acctbal desc,
+ n_name,
+ s_name,
+ p_partkey
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q4.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q4.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q4.tql
index 760d183..2ab4598 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q4.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q4.tql
@@ -1,7 +1,21 @@
select
- *
-from
- lineitem
-where
- l_orderkey = o_orderkey and
- l_commitdate < l_receiptdate
\ No newline at end of file
+ o_orderpriority,
+ count(*) as order_count
+from
+ orders
+where
+ o_orderdate >= '1993-07-01'
+ and o_orderdate < '1993-10-01'
+ and exists (
+ select
+ *
+ from
+ lineitem
+ where
+ l_orderkey = o_orderkey
+ and l_commitdate < l_receiptdate
+ )
+group by
+ o_orderpriority
+order by
+ o_orderpriority;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q7.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q7.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q7.tql
index 9cf48d6..a7d0f79 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q7.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q7.tql
@@ -1,21 +1,38 @@
select
- n1.n_name as supp_nation,
- n2.n_name as cust_nation,
- extract(year from l_shipdate) as l_year,
- l_extendedprice * (1 - l_discount) as volume
-from
- supplier,
- lineitem,
- orders,
- customer,
- nation n1,
- nation n2
-where
- s_suppkey = l_suppkey and
- o_orderkey = l_orderkey and
- c_custkey = o_custkey and
- s_nationkey = n1.n_nationkey and
- c_nationkey = n2.n_nationkey and
- ( (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY') or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE') ) and
- l_shipdate > '1995-01-01' and
- l_shipdate < '1996-12-31' as shipping
\ No newline at end of file
+ supp_nation,
+ cust_nation,
+ l_year,
+ sum(volume) as revenue
+from (
+ select
+ n1.n_name as supp_nation,
+ n2.n_name as cust_nation,
+ l_shipdate as l_year,
+ l_extendedprice * (1 - l_discount) as volume
+ from
+ supplier,
+ lineitem,
+ orders,
+ customer,
+ nation n1,
+ nation n2
+ where
+ s_suppkey = l_suppkey
+ and o_orderkey = l_orderkey
+ and c_custkey = o_custkey
+ and s_nationkey = n1.n_nationkey
+ and c_nationkey = n2.n_nationkey
+ and (
+ (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY')
+ or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE')
+ )
+ and '1995-01-01' < l_shipdate and l_shipdate < '1996-12-31'
+ ) as shipping
+group by
+ supp_nation,
+ cust_nation,
+ l_year
+order by
+ supp_nation,
+ cust_nation,
+ l_year;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q8.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q8.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q8.tql
index 3dbde76..9e96d4d 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q8.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q8.tql
@@ -1,25 +1,37 @@
select
- extract(year from o_orderdate) as o_year,
- l_extendedprice * (1-l_discount) as volume,
- n2.n_name as nation
-from
- part,
- supplier,
- lineitem,
- orders,
- customer,
- nation n1,
- nation n2,
- region
-where
- p_partkey = l_partkey and
- s_suppkey = l_suppkey and
- l_orderkey = o_orderkey and
- o_custkey = c_custkey and
- c_nationkey = n1.n_nationkey and
- n1.n_regionkey = r_regionkey and
- r_name = 'AMERICA' and
- s_nationkey = n2.n_nationkey and
- o_orderdate > '1995-01-01' and
- o_orderdate < '1996-12-31' and
- p_type = 'ECONOMY ANODIZED STEEL' as all_nations
\ No newline at end of file
+ o_year,
+ sum(case
+ when nation = 'BRAZIL'
+ then volume
+ else 0
+ end) / sum(volume) as mkt_share
+from (
+ select
+ o_orderdate as o_year,
+ l_extendedprice * (1-l_discount) as volume,
+ n2.n_name as nation
+ from
+ part,
+ supplier,
+ lineitem,
+ orders,
+ customer,
+ nation n1,
+ nation n2,
+ region
+ where
+ p_partkey = l_partkey
+ and s_suppkey = l_suppkey
+ and l_orderkey = o_orderkey
+ and o_custkey = c_custkey
+ and c_nationkey = n1.n_nationkey
+ and n1.n_regionkey = r_regionkey
+ and r_name = 'AMERICA'
+ and s_nationkey = n2.n_nationkey
+ and '1995-01-01' < o_orderdate and o_orderdate <'1996-12-31'
+ and p_type = 'ECONOMY ANODIZED STEEL'
+ ) as all_nations
+group by
+ o_year
+order by
+ o_year;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/benchmark/tpch/q9.tql
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/benchmark/tpch/q9.tql b/tajo-core/tajo-core-backend/benchmark/tpch/q9.tql
index 396c9c3..c22523e 100644
--- a/tajo-core/tajo-core-backend/benchmark/tpch/q9.tql
+++ b/tajo-core/tajo-core-backend/benchmark/tpch/q9.tql
@@ -1,19 +1,31 @@
-select
- n_name as nation,
- extract(year from o_orderdate) as o_year,
- l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount
-from
- part,
- supplier,
- lineitem,
- partsupp,
- orders,
- nation
-where
- s_suppkey = l_suppkey and
- ps_suppkey = l_suppkey and
- ps_partkey = l_partkey and
- p_partkey = l_partkey and
- o_orderkey = l_orderkey and
- s_nationkey = n_nationkey and
- p_name like '%green%' ) as profit
\ No newline at end of file
+select
+ nation,
+ o_year,
+ sum(amount) as sum_profit
+ from (
+ select
+ n_name as nation,
+ o_orderdate as o_year,
+ l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount
+ from
+ part,
+ supplier,
+ lineitem,
+ partsupp,
+ orders,
+ nation
+ where
+ s_suppkey = l_suppkey
+ and ps_suppkey = l_suppkey
+ and ps_partkey = l_partkey
+ and p_partkey = l_partkey
+ and o_orderkey = l_orderkey
+ and s_nationkey = n_nationkey
+ and p_name like '%[COLOR]%'
+ ) as profit
+group by
+ nation,
+ o_year
+order by
+ nation,
+ o_year desc;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java
new file mode 100644
index 0000000..be1852d
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/DataChannel.java
@@ -0,0 +1,198 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo;
+
+import com.google.common.base.Preconditions;
+import org.apache.tajo.catalog.Column;
+import org.apache.tajo.catalog.Schema;
+import org.apache.tajo.master.ExecutionBlock;
+
+import static org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.*;
+
+public class DataChannel {
+ private ExecutionBlockId srcId;
+ private ExecutionBlockId targetId;
+ private TransmitType transmitType = TransmitType.PULL_TRANSMIT;
+ private PartitionType partitionType;
+ private Integer partitionNum = 1;
+ private Column[] key;
+
+ private Schema schema;
+
+ private StoreType storeType = StoreType.CSV;
+
+ public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId) {
+ this.srcId = srcId;
+ this.targetId = targetId;
+ }
+
+ public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId, PartitionType partitionType) {
+ this(srcId, targetId);
+ this.partitionType = partitionType;
+ }
+
+ public DataChannel(ExecutionBlock src, ExecutionBlock target, PartitionType partitionType, int partNum) {
+ this(src.getId(), target.getId(), partitionType, partNum);
+ setSchema(src.getPlan().getOutSchema());
+ }
+
+ public DataChannel(ExecutionBlockId srcId, ExecutionBlockId targetId, PartitionType partitionType, int partNum) {
+ this(srcId, targetId, partitionType);
+ this.partitionNum = partNum;
+ }
+
+ public DataChannel(DataChannelProto proto) {
+ this.srcId = new ExecutionBlockId(proto.getSrcId());
+ this.targetId = new ExecutionBlockId(proto.getTargetId());
+ this.transmitType = proto.getTransmitType();
+ this.partitionType = proto.getPartitionType();
+ if (proto.hasSchema()) {
+ this.setSchema(new Schema(proto.getSchema()));
+ }
+ if (proto.getPartitionKeyCount() > 0) {
+ key = new Column[proto.getPartitionKeyCount()];
+ for (int i = 0; i < proto.getPartitionKeyCount(); i++) {
+ key[i] = new Column(proto.getPartitionKey(i));
+ }
+ } else {
+ key = new Column[] {};
+ }
+ if (proto.hasPartitionNum()) {
+ this.partitionNum = proto.getPartitionNum();
+ }
+ }
+
+ public ExecutionBlockId getSrcId() {
+ return srcId;
+ }
+
+ public ExecutionBlockId getTargetId() {
+ return targetId;
+ }
+
+ public PartitionType getPartitionType() {
+ return partitionType;
+ }
+
+ public TransmitType getTransmitType() {
+ return this.transmitType;
+ }
+
+ public void setTransmitType(TransmitType transmitType) {
+ this.transmitType = transmitType;
+ }
+
+ public void setPartition(PartitionType partitionType, Column [] keys, int numPartitions) {
+ Preconditions.checkArgument(keys.length >= 0, "At least one partition key must be specified.");
+ Preconditions.checkArgument(numPartitions > 0, "The number of partitions must be positive: %s", numPartitions);
+
+ this.partitionType = partitionType;
+ this.key = keys;
+ this.partitionNum = numPartitions;
+ }
+
+ public void setPartitionType(PartitionType partitionType) {
+ this.partitionType = partitionType;
+ }
+
+ public boolean hasPartitionKey() {
+ return key != null;
+ }
+
+ public void setPartitionKey(Column [] key) {
+ this.key = key;
+ }
+
+ public Column [] getPartitionKey() {
+ return this.key;
+ }
+
+ public void setPartitionNum(int partNum) {
+ this.partitionNum = partNum;
+ }
+
+ public int getPartitionNum() {
+ return partitionNum;
+ }
+
+ public boolean hasStoreType() {
+ return this.storeType != null;
+ }
+
+ public void setStoreType(StoreType storeType) {
+ this.storeType = storeType;
+ }
+
+ public StoreType getStoreType() {
+ return storeType;
+ }
+
+ public DataChannelProto getProto() {
+ DataChannelProto.Builder builder = DataChannelProto.newBuilder();
+ builder.setSrcId(srcId.getProto());
+ builder.setTargetId(targetId.getProto());
+ if (transmitType != null) {
+ builder.setTransmitType(transmitType);
+ }
+ builder.setPartitionType(partitionType);
+ if (schema != null) {
+ builder.setSchema(schema.getProto());
+ }
+ if (key != null) {
+ for (Column column : key) {
+ builder.addPartitionKey(column.getProto());
+ }
+ }
+ if (partitionNum != null) {
+ builder.setPartitionNum(partitionNum);
+ }
+ return builder.build();
+ }
+
+ public void setSchema(Schema schema) {
+ this.schema = schema;
+ }
+
+ public Schema getSchema() {
+ return schema;
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[").append(srcId.getQueryId()).append("] ");
+ sb.append(srcId.getId()).append(" => ").append(targetId.getId());
+ sb.append(" (type=").append(partitionType);
+ if (hasPartitionKey()) {
+ sb.append(", key=");
+ boolean first = true;
+ for (Column column : getPartitionKey()) {
+ if (first) {
+ first = false;
+ } else {
+ sb.append(",");
+ }
+ sb.append(column.getColumnName());
+ }
+ sb.append(", num=").append(partitionNum);
+ }
+ sb.append(")");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java
index 8ae2f2b..7eac43d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/TaskAttemptContext.java
@@ -54,6 +54,7 @@ public class TaskAttemptContext {
private boolean stopped = false;
private boolean interQuery = false;
private Path outputPath;
+ private DataChannel dataChannel;
public TaskAttemptContext(TajoConf conf, final QueryUnitAttemptId queryId,
final Fragment[] fragments,
@@ -90,6 +91,14 @@ public class TaskAttemptContext {
LOG.info("Query status of " + getTaskId() + " is changed to " + state);
}
+ public void setDataChannel(DataChannel dataChannel) {
+ this.dataChannel = dataChannel;
+ }
+
+ public DataChannel getDataChannel() {
+ return dataChannel;
+ }
+
public boolean hasResultStats() {
return resultStats != null;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
index 3fc41dd..e6bd733 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
@@ -31,13 +31,7 @@ import org.apache.tajo.exception.InternalException;
import java.util.*;
public class EvalTreeUtil {
- public static void changeColumnRef(EvalNode node, Column oldName,
- Column newName) {
- node.postOrder(new ChangeColumnRefVisitor(oldName.getQualifiedName(),
- newName.getQualifiedName()));
- }
-
- public static void changeColumnRef(EvalNode node, String oldName,
+ public static void changeColumnRef(EvalNode node, String oldName,
String newName) {
node.postOrder(new ChangeColumnRefVisitor(oldName, newName));
}
@@ -131,7 +125,7 @@ public class EvalTreeUtil {
case FIELD:
FieldEval fieldEval = (FieldEval) expr;
- return SchemaUtil.newNoNameSchema(inputSchema.getColumn(fieldEval.getName()).getDataType());
+ return SchemaUtil.newNoNameSchema(inputSchema.getColumnByFQN(fieldEval.getName()).getDataType());
default:
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/FieldEval.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
index cd9c48d..87d6077 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/eval/FieldEval.java
@@ -47,7 +47,7 @@ public class FieldEval extends EvalNode implements Cloneable {
fieldId = schema.getColumnId(column.getQualifiedName());
} else {
if(schema.getColumnNum() != 0) {
- String schemaColQualName = schema.getColumn(0).getTableName() +
+ String schemaColQualName = schema.getColumn(0).getQualifier() +
"." + column.getColumnName();
fieldId = schema.getColumnId(schemaColQualName);
} else {
@@ -86,7 +86,7 @@ public class FieldEval extends EvalNode implements Cloneable {
}
public String getTableId() {
- return column.getTableName();
+ return column.getQualifier();
}
public String getColumnName() {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
index 96e67f2..133fc50 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java
@@ -21,7 +21,6 @@ package org.apache.tajo.engine.parser;
import com.google.common.base.Preconditions;
import org.antlr.v4.runtime.ANTLRInputStream;
import org.antlr.v4.runtime.CommonTokenStream;
-import org.antlr.v4.runtime.misc.NotNull;
import org.antlr.v4.runtime.tree.TerminalNode;
import org.apache.tajo.algebra.*;
import org.apache.tajo.algebra.Aggregation.GroupType;
@@ -82,7 +81,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
OpType operatorType;
Expr left;
- for (int i = 1; i < ctx.getChildCount();) {
+ for (int i = 1; i < ctx.getChildCount(); i++) {
int idx = i;
boolean distinct = true;
@@ -110,7 +109,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
left = current;
current = new SetOperation(operatorType, left, right, distinct);
- i+=idx;
+ i = idx;
}
}
@@ -254,7 +253,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
public RelationList visitFrom_clause(SQLParser.From_clauseContext ctx) {
Expr [] relations = new Expr[ctx.table_reference_list().table_reference().size()];
for (int i = 0; i < relations.length; i++) {
- relations[i] = visit(ctx.table_reference_list().table_reference(i));
+ relations[i] = visitTable_reference(ctx.table_reference_list().table_reference(i));
}
return new RelationList(relations);
}
@@ -391,7 +390,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
public Expr visitTable_primary(SQLParser.Table_primaryContext ctx) {
if (ctx.table_or_query_name() != null) {
Relation relation = new Relation(ctx.table_or_query_name().getText());
- if (ctx.AS() != null) {
+ if (ctx.alias != null) {
relation.setAlias(ctx.alias.getText());
}
return relation;
@@ -656,7 +655,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
}
}
- @Override public FunctionExpr visitAggregate_function(@NotNull SQLParser.Aggregate_functionContext ctx) {
+ @Override public FunctionExpr visitAggregate_function( SQLParser.Aggregate_functionContext ctx) {
if (ctx.COUNT() != null && ctx.MULTIPLY() != null) {
return new CountRowsFunctionExpr();
} else {
@@ -664,7 +663,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor<Expr> {
}
}
- @Override public FunctionExpr visitGeneral_set_function(@NotNull SQLParser.General_set_functionContext ctx) {
+ @Override public FunctionExpr visitGeneral_set_function(SQLParser.General_set_functionContext ctx) {
String signature = ctx.set_function_type().getText();
boolean distinct = checkIfExist(ctx.set_qualifier()) && checkIfExist(ctx.set_qualifier().DISTINCT()) ? true : false;
Expr param = visitBoolean_value_expression(ctx.boolean_value_expression());
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
index 9dba5e1..78422b0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java
@@ -33,6 +33,7 @@ public interface AlgebraVisitor<T1, T2> {
T2 visitExcept(T1 ctx, Stack<OpType> stack, SetOperation expr) throws PlanningException;
T2 visitIntersect(T1 ctx, Stack<OpType> stack, SetOperation expr) throws PlanningException;
T2 visitRelationList(T1 ctx, Stack<OpType> stack, RelationList expr) throws PlanningException;
+ T2 visitTableSubQuery(T1 ctx, Stack<OpType> stack, TableSubQuery expr) throws PlanningException;
T2 visitRelation(T1 ctx, Stack<OpType> stack, Relation expr) throws PlanningException;
T2 visitCreateTable(T1 ctx, Stack<OpType> stack, CreateTable expr) throws PlanningException;
T2 visitDropTable(T1 ctx, Stack<OpType> stack, DropTable expr) throws PlanningException;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
index 4c06f28..bac146c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java
@@ -79,6 +79,9 @@ public abstract class BaseAlgebraVisitor<T1, T2> implements AlgebraVisitor<T1, T
case RelationList:
current = visitRelationList(ctx, stack, (RelationList) expr);
break;
+ case TableSubQuery:
+ current = visitTableSubQuery(ctx, stack, (TableSubQuery) expr);
+ break;
case Relation:
current = visitRelation(ctx, stack, (Relation) expr);
break;
@@ -195,6 +198,14 @@ public abstract class BaseAlgebraVisitor<T1, T2> implements AlgebraVisitor<T1, T
}
@Override
+ public T2 visitTableSubQuery(T1 ctx, Stack<OpType> stack, TableSubQuery expr) throws PlanningException {
+ stack.push(expr.getType());
+ T2 child = visitChild(ctx, stack, expr.getSubQuery());
+ stack.pop();
+ return child;
+ }
+
+ @Override
public T2 visitRelation(T1 ctx, Stack<OpType> stack, Relation expr) throws PlanningException {
return null;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java
index 52d2a0a..b58b05e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java
@@ -76,6 +76,9 @@ public class BasicLogicalPlanVisitor<T> implements LogicalPlanVisitor<T> {
case INTERSECT:
current = visitIntersect(plan, (IntersectNode) node, stack, data);
break;
+ case TABLE_SUBQUERY:
+ current = visitTableSubQuery(plan, (TableSubQueryNode) node, stack, data);
+ break;
case SCAN:
current = visitScan(plan, (ScanNode) node, stack, data);
break;
@@ -184,6 +187,15 @@ public class BasicLogicalPlanVisitor<T> implements LogicalPlanVisitor<T> {
}
@Override
+ public LogicalNode visitTableSubQuery(LogicalPlan plan, TableSubQueryNode node, Stack<LogicalNode> stack, T data)
+ throws PlanningException {
+ stack.push(node);
+ visitChild(plan, node.getSubQuery(), stack, data);
+ stack.pop();
+ return node;
+ }
+
+ @Override
public LogicalNode visitScan(LogicalPlan plan, ScanNode node, Stack<LogicalNode> stack, T data)
throws PlanningException {
return node;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java
new file mode 100644
index 0000000..d572494
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/ExplainLogicalPlanVisitor.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner;
+
+import org.apache.tajo.engine.planner.logical.*;
+
+import java.util.Stack;
+
+/**
+ * It returns a list of node plan strings.
+ */
+public class ExplainLogicalPlanVisitor extends BasicLogicalPlanVisitor<ExplainLogicalPlanVisitor.Context> {
+
+ public static class Context {
+ public int maxDepth = -1;
+ public int depth = 0;
+ public Stack<DepthString> explains = new Stack<DepthString>();
+
+ public void add(int depth, PlanString planString) {
+ maxDepth = Math.max(maxDepth, depth);
+ explains.push(new DepthString(depth, planString));
+ }
+
+ public int getMaxDepth() {
+ return this.maxDepth;
+ }
+
+ public Stack<DepthString> getExplains() {
+ return explains;
+ }
+ }
+
+ public static class DepthString {
+ private int depth;
+ private PlanString planStr;
+
+ DepthString(int depth, PlanString planStr) {
+ this.depth = depth;
+ this.planStr = planStr;
+ }
+
+ public int getDepth() {
+ return depth;
+ }
+
+ public PlanString getPlanString() {
+ return planStr;
+ }
+ }
+
+ public Context getBlockPlanStrings(LogicalPlan plan, String block) throws PlanningException {
+ Stack<LogicalNode> stack = new Stack<LogicalNode>();
+ Context explainContext = new Context();
+ visitChild(plan, plan.getBlock(block).getRoot(), stack, explainContext);
+ return explainContext;
+ }
+
+ @Override
+ public LogicalNode visitRoot(LogicalPlan plan, LogicalRootNode node, Stack<LogicalNode> stack, Context data)
+ throws PlanningException {
+ return visitChild(plan, node.getChild(), stack, data);
+ }
+
+ public LogicalNode visitProjection(LogicalPlan plan, ProjectionNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ return visitUnaryNode(plan, node, stack, context);
+ }
+
+ @Override
+ public LogicalNode visitLimit(LogicalPlan plan, LimitNode node, Stack<LogicalNode> stack, Context context)
+ throws PlanningException {
+ return visitUnaryNode(plan, node, stack, context);
+ }
+
+ public LogicalNode visitSort(LogicalPlan plan, SortNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ return visitUnaryNode(plan, node, stack, context);
+ }
+
+ public LogicalNode visitGroupBy(LogicalPlan plan, GroupbyNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ return visitUnaryNode(plan, node, stack, context);
+ }
+
+ private LogicalNode visitUnaryNode(LogicalPlan plan, UnaryNode node, Stack<LogicalNode> stack, Context context)
+ throws PlanningException {
+ context.depth++;
+ stack.push(node);
+ visitChild(plan, node.getChild(), stack, context);
+ context.depth--;
+ context.add(context.depth, node.getPlanString());
+ return node;
+ }
+
+ private LogicalNode visitBinaryNode(LogicalPlan plan, BinaryNode node, Stack<LogicalNode> stack, Context context)
+ throws PlanningException {
+ context.depth++;
+ stack.push(node);
+ visitChild(plan, node.getLeftChild(), stack, context);
+ visitChild(plan, node.getRightChild(), stack, context);
+ stack.pop();
+ context.depth--;
+ context.add(context.depth, node.getPlanString());
+ return node;
+ }
+
+ public LogicalNode visitFilter(LogicalPlan plan, SelectionNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ return visitUnaryNode(plan, node, stack, context);
+ }
+
+ @Override
+ public LogicalNode visitJoin(LogicalPlan plan, JoinNode node, Stack<LogicalNode> stack, Context context)
+ throws PlanningException {
+ return visitBinaryNode(plan, node, stack, context);
+ }
+
+ @Override
+ public LogicalNode visitUnion(LogicalPlan plan, UnionNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ return visitBinaryNode(plan, node, stack, context);
+ }
+
+ @Override
+ public LogicalNode visitExcept(LogicalPlan plan, ExceptNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ return visitBinaryNode(plan, node, stack, context);
+ }
+
+ @Override
+ public LogicalNode visitIntersect(LogicalPlan plan, IntersectNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ return visitBinaryNode(plan, node, stack, context);
+ }
+
+ @Override
+ public LogicalNode visitTableSubQuery(LogicalPlan plan, TableSubQueryNode node, Stack<LogicalNode> stack,
+ Context context) throws PlanningException {
+ context.depth++;
+ stack.push(node);
+ super.visitTableSubQuery(plan, node, stack, context);
+ stack.pop();
+ context.depth--;
+ context.add(context.depth, node.getPlanString());
+
+ return node;
+ }
+
+ @Override
+ public LogicalNode visitScan(LogicalPlan plan, ScanNode node, Stack<LogicalNode> stack, Context context)
+ throws PlanningException {
+ context.add(context.depth, node.getPlanString());
+ return node;
+ }
+
+ @Override
+ public LogicalNode visitStoreTable(LogicalPlan plan, StoreTableNode node, Stack<LogicalNode> stack, Context context) throws PlanningException {
+ return visitUnaryNode(plan, node, stack, context);
+ }
+
+ @Override
+ public LogicalNode visitInsert(LogicalPlan plan, InsertNode node, Stack<LogicalNode> stack, Context context) throws PlanningException {
+ context.depth++;
+ stack.push(node);
+ visitChild(plan, node.getSubQuery(), stack, context);
+ stack.pop();
+ context.depth--;
+ context.add(context.depth, node.getPlanString());
+ return node;
+ }
+
+ public static String printDepthString(int maxDepth, DepthString planStr) {
+ StringBuilder output = new StringBuilder();
+ String pad = new String(new char[planStr.getDepth() * 3]).replace('\0', ' ');
+ output.append(pad + planStr.getPlanString().getTitle()).append("\n");
+
+ for (String str : planStr.getPlanString().getExplanations()) {
+ output.append(pad).append(" => ").append(str).append("\n");
+ }
+
+ for (String str : planStr.getPlanString().getDetails()) {
+ output.append(pad).append(" => ").append(str).append("\n");
+ }
+ return output.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/InsertNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/InsertNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/InsertNode.java
index f85fae5..65c0dfa 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/InsertNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/InsertNode.java
@@ -180,4 +180,16 @@ public class InsertNode extends LogicalNode implements Cloneable {
public void postOrder(LogicalNodeVisitor visitor) {
visitor.visit(this);
}
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planString = new PlanString("INSERT");
+ planString.addExplan(" INTO ");
+ if (hasTargetTable()) {
+ planString.addExplan(getTargetTable().getName());
+ } else {
+ planString.addExplan("LOCATION " + path);
+ }
+ return planString;
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
index ed88cbc..91f3e7c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
@@ -27,6 +27,8 @@ import org.apache.tajo.catalog.Schema;
import org.apache.tajo.engine.eval.EvalNode;
import org.apache.tajo.engine.eval.EvalTreeUtil;
import org.apache.tajo.engine.eval.EvalType;
+import org.apache.tajo.engine.planner.graph.DirectedGraphVisitor;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
import org.apache.tajo.engine.planner.logical.*;
import java.util.*;
@@ -52,7 +54,7 @@ public class LogicalPlan {
/** a map from between a block name to a block plan */
private Map<String, QueryBlock> queryBlocks = new LinkedHashMap<String, QueryBlock>();
private Map<LogicalNode, QueryBlock> queryBlockByNode = new HashMap<LogicalNode, QueryBlock>();
- private QueryBlockGraph queryBlockGraph = new QueryBlockGraph();
+ private SimpleDirectedGraph<QueryBlock, BlockEdge> blockGraph = new SimpleDirectedGraph<QueryBlock, BlockEdge>();
private Set<LogicalNode> visited = new HashSet<LogicalNode>();
public LogicalPlan(LogicalPlanner planner) {
@@ -111,18 +113,22 @@ public class LogicalPlan {
}
}
- public void connectBlocks(QueryBlock srcBlock, QueryBlock targetBlock, QueryBlockGraph.BlockType type) {
- queryBlockGraph.connectBlocks(srcBlock.getName(), targetBlock.getName(), type);
+ public void connectBlocks(QueryBlock srcBlock, QueryBlock targetBlock, BlockType type) {
+ blockGraph.connect(srcBlock, targetBlock, new BlockEdge(srcBlock, targetBlock, type));
}
- public Collection<QueryBlockGraph.BlockEdge> getConnectedBlocks(String blockName) {
- return queryBlockGraph.getBlockEdges(blockName);
+ public List<QueryBlock> getChildBlocks(QueryBlock block) {
+ return blockGraph.getChilds(block);
}
public Collection<QueryBlock> getQueryBlocks() {
return queryBlocks.values();
}
+ public SimpleDirectedGraph<QueryBlock, BlockEdge> getBlockGraph() {
+ return blockGraph;
+ }
+
public boolean postVisit(String blockName, LogicalNode node, Stack<OpType> path) {
if (visited.contains(node)) {
return false;
@@ -142,7 +148,7 @@ public class LogicalPlan {
break;
case GROUP_BY:
- block.resolveGrouping();
+ block.needToResolveGrouping();
break;
case SELECTION:
@@ -152,6 +158,11 @@ public class LogicalPlan {
case INSERT:
block.setInsertNode((InsertNode) node);
break;
+
+ case TABLE_SUBQUERY:
+ TableSubQueryNode tableSubQueryNode = (TableSubQueryNode) node;
+ block.addRelation(tableSubQueryNode);
+ break;
}
@@ -172,7 +183,7 @@ public class LogicalPlan {
throws VerifyException {
QueryBlock block = queryBlocks.get(blockName);
- ScanNode relationOp = block.getRelation(relName);
+ RelationNode relationOp = block.getRelation(relName);
// if a column name is outside of this query block
if (relationOp == null) {
@@ -200,7 +211,7 @@ public class LogicalPlan {
} catch (CloneNotSupportedException e) {
e.printStackTrace();
}
- String tableName = relationOp.getTableId();
+ String tableName = relationOp.getTableName();
column.setName(tableName + "." + column.getColumnName());
return column;
@@ -218,11 +229,11 @@ public class LogicalPlan {
Column candidate;
if (columnRef.hasTableName()) {
- candidate = node.getOutSchema().getColumn(columnRef.getCanonicalName());
+ candidate = node.getOutSchema().getColumnByFQN(columnRef.getCanonicalName());
if (candidate == null) { // If not found, try to find the column with alias name
- String tableName = getBlock(blockName).getRelation(columnRef.getTableName()).getTableId();
- candidate = node.getOutSchema().getColumn(tableName + "." + columnRef.getName());
+ String tableName = getBlock(blockName).getRelation(columnRef.getTableName()).getTableName();
+ candidate = node.getOutSchema().getColumnByFQN(tableName + "." + columnRef.getName());
}
candidates.add(candidate);
@@ -266,7 +277,7 @@ public class LogicalPlan {
Column candidate;
// Try to find a column from the current query block
- for (ScanNode rel : queryBlocks.get(blockName).getRelations()) {
+ for (RelationNode rel : queryBlocks.get(blockName).getRelations()) {
candidate = findColumnFromRelationOp(rel, name);
if (candidate != null) {
@@ -276,7 +287,7 @@ public class LogicalPlan {
} catch (CloneNotSupportedException e) {
e.printStackTrace();
}
- candidate.setName(rel.getTableId() + "." + candidate.getColumnName());
+ candidate.setName(rel.getTableName() + "." + candidate.getColumnName());
}
candidates.add(candidate);
if (candidates.size() > 0) {
@@ -290,7 +301,7 @@ public class LogicalPlan {
// for each block
Outer:
for (QueryBlock block : queryBlocks.values()) {
- for (ScanNode rel : block.getRelations()) {
+ for (RelationNode rel : block.getRelations()) {
candidate = findColumnFromRelationOp(rel, name);
if (candidate != null) {
@@ -300,7 +311,7 @@ public class LogicalPlan {
} catch (CloneNotSupportedException e) {
e.printStackTrace();
}
- candidate.setName(rel.getTableId() + "." + candidate.getColumnName());
+ candidate.setName(rel.getTableName() + "." + candidate.getColumnName());
}
candidates.add(candidate);
if (candidates.size() > 0)
@@ -311,7 +322,7 @@ public class LogicalPlan {
}
if (candidates.isEmpty()) {
- throw new VerifyException("ERROR: no such a column name "+ name);
+ throw new VerifyException("ERROR: no such a column '"+ name + "'");
} else if (candidates.size() > 1) {
throw new VerifyException("ERROR: column name "+ name + " is ambiguous");
}
@@ -319,7 +330,7 @@ public class LogicalPlan {
return candidates.get(0);
}
- private Column findColumnFromRelationOp(ScanNode relation, String name) throws VerifyException {
+ private Column findColumnFromRelationOp(RelationNode relation, String name) throws VerifyException {
Column candidate = relation.getTableSchema().getColumnByName(name);
if (candidate != null) {
try {
@@ -328,7 +339,7 @@ public class LogicalPlan {
throw new RuntimeException(e);
}
if (!isVirtualRelation(relation.getCanonicalName())) {
- candidate.setName(relation.getTableId() + "." + name);
+ candidate.setName(relation.getTableName() + "." + name);
}
return candidate;
@@ -345,7 +356,7 @@ public class LogicalPlan {
private String blockName;
private LogicalNode rootNode;
private NodeType rootType;
- private Map<String, ScanNode> relations = new HashMap<String, ScanNode>();
+ private Map<String, RelationNode> relations = new HashMap<String, RelationNode>();
private Projection projection;
private boolean resolvedGrouping = true;
@@ -384,8 +395,8 @@ public class LogicalPlan {
queryBlockByNode.put(blockRoot, this);
}
- public LogicalNode getRoot() {
- return rootNode;
+ public <T extends LogicalNode> T getRoot() {
+ return (T) rootNode;
}
public NodeType getRootType() {
@@ -396,15 +407,15 @@ public class LogicalPlan {
return relations.containsKey(name);
}
- public void addRelation(ScanNode relation) {
+ public void addRelation(RelationNode relation) {
relations.put(relation.getCanonicalName(), relation);
}
- public ScanNode getRelation(String name) {
+ public RelationNode getRelation(String name) {
return relations.get(name);
}
- public Collection<ScanNode> getRelations() {
+ public Collection<RelationNode> getRelations() {
return this.relations.values();
}
@@ -440,7 +451,7 @@ public class LogicalPlan {
return this.resolvedGrouping;
}
- public void resolveGrouping() {
+ public void needToResolveGrouping() {
this.resolvedGrouping = true;
}
@@ -524,7 +535,7 @@ public class LogicalPlan {
}
// If all columns are projected and do not include any expression
- if (projection.isAllProjected() && node instanceof ScanNode) {
+ if (projection.isAllProjected() && node instanceof RelationNode) {
targetListManager = new TargetListManager(LogicalPlan.this, PlannerUtil.schemaToTargets(node.getInSchema()));
targetListManager.setEvaluatedAll();
@@ -536,7 +547,7 @@ public class LogicalPlan {
for (int i = 0; i < targetListManager.size(); i++) {
if (targetListManager.getTarget(i) == null) {
try {
- targetListManager.updateTarget(i,planner.createTarget(LogicalPlan.this, blockName,
+ targetListManager.updateTarget(i, planner.createTarget(LogicalPlan.this, blockName,
projection.getTargets()[i]));
} catch (VerifyException e) {
}
@@ -552,7 +563,7 @@ public class LogicalPlan {
if (canTargetEvaluated(i, node)) {
- if (node instanceof ScanNode) { // for scan node
+ if (node instanceof RelationNode) { // for scan node
if (expr.getType() == EvalType.FIELD) {
targetListManager.setEvaluated(i);
if (targetListManager.getTarget(i).hasAlias()) {
@@ -631,4 +642,76 @@ public class LogicalPlan {
return targetListManager.getTargets();
}
}
+
+ public String getQueryGraphAsString() {
+ StringBuilder sb = new StringBuilder();
+
+ sb.append("-----------------------------\n");
+ sb.append("Query Block Graph\n");
+ sb.append("-----------------------------\n");
+ sb.append(blockGraph.toStringGraph(getRootBlock()));
+ sb.append("-----------------------------\n");
+
+ sb.append("\n");
+
+ sb.append(getLogicalPlanAsString());
+
+ return sb.toString();
+ }
+
+ public String getLogicalPlanAsString() {
+ ExplainLogicalPlanVisitor explain = new ExplainLogicalPlanVisitor();
+
+ StringBuilder explains = new StringBuilder();
+ try {
+ ExplainLogicalPlanVisitor.Context explainContext = explain.getBlockPlanStrings(this, ROOT_BLOCK);
+ while(!explainContext.explains.empty()) {
+ explains.append(
+ ExplainLogicalPlanVisitor.printDepthString(explainContext.getMaxDepth(), explainContext.explains.pop()));
+ }
+ } catch (PlanningException e) {
+ e.printStackTrace();
+ }
+
+ return explains.toString();
+ }
+
+ @Override
+ public String toString() {
+ return getQueryGraphAsString();
+ }
+
+ public static enum BlockType {
+ TableSubQuery,
+ ScalarSubQuery
+ }
+
+ public static class BlockEdge {
+ private String childName;
+ private String parentName;
+ private BlockType blockType;
+
+
+ public BlockEdge(String childName, String parentName, BlockType blockType) {
+ this.childName = childName;
+ this.parentName = parentName;
+ this.blockType = blockType;
+ }
+
+ public BlockEdge(QueryBlock child, QueryBlock parent, BlockType blockType) {
+ this(child.getName(), parent.getName(), blockType);
+ }
+
+ public String getParentName() {
+ return parentName;
+ }
+
+ public String getChildName() {
+ return childName;
+ }
+
+ public BlockType getBlockType() {
+ return blockType;
+ }
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java
index c41c534..ca5a8f0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java
@@ -43,6 +43,8 @@ public interface LogicalPlanVisitor <T> {
throws PlanningException;
LogicalNode visitIntersect(LogicalPlan plan, IntersectNode node, Stack<LogicalNode> stack, T data)
throws PlanningException;
+ LogicalNode visitTableSubQuery(LogicalPlan plan, TableSubQueryNode node, Stack<LogicalNode> stack, T data)
+ throws PlanningException;
LogicalNode visitScan(LogicalPlan plan, ScanNode node, Stack<LogicalNode> stack, T data)
throws PlanningException;
LogicalNode visitStoreTable(LogicalPlan plan, StoreTableNode node, Stack<LogicalNode> stack, T data)
[4/7] TAJO-184: Refactor GlobalPlanner and global plan data
structure. (hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
index 7c374d2..d5e51ef 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalPlanner.java
@@ -18,681 +18,432 @@
package org.apache.tajo.master;
+import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tajo.ExecutionBlockId;
-import org.apache.tajo.QueryId;
-import org.apache.tajo.QueryIdFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.catalog.*;
-import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.FromTable;
-import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.planner.*;
import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.planner.logical.*;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
import org.apache.tajo.storage.AbstractStorageManager;
-import org.apache.tajo.util.TajoIdUtils;
import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
-import static org.apache.tajo.conf.TajoConf.ConfVars;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType.*;
public class GlobalPlanner {
private static Log LOG = LogFactory.getLog(GlobalPlanner.class);
private TajoConf conf;
private AbstractStorageManager sm;
- private QueryId queryId;
- public GlobalPlanner(final TajoConf conf,
- final AbstractStorageManager sm,
- final EventHandler eventHandler)
+ public GlobalPlanner(final TajoConf conf, final AbstractStorageManager sm)
throws IOException {
this.conf = conf;
this.sm = sm;
}
+ public class GlobalPlanContext {
+ MasterPlan plan;
+ Set<String> broadcastTables = new HashSet<String>();
+ LogicalNode topmost;
+ LogicalNode lastRepartionableNode;
+ ExecutionBlock topMostLeftExecBlock;
+ ExecutionBlock topMostRightExecBlock;
+ }
+
/**
* Builds a master plan from the given logical plan.
- * @param queryId
- * @param rootNode
- * @return
- * @throws IOException
*/
- public MasterPlan build(QueryId queryId, LogicalRootNode rootNode)
- throws IOException {
- this.queryId = queryId;
-
- String outputTableName = null;
- if (rootNode.getChild().getType() == NodeType.STORE) {
- // create table queries are executed by the master
- StoreTableNode storeTableNode = (StoreTableNode) rootNode.getChild();
- outputTableName = storeTableNode.getTableName();
+ public void build(MasterPlan masterPlan)
+ throws IOException, PlanningException {
+
+ NewPlanner planner = new NewPlanner();
+ GlobalPlanContext globalPlanContext = new GlobalPlanContext();
+ globalPlanContext.plan = masterPlan;
+ LOG.info(masterPlan.getLogicalPlan());
+
+ LogicalNode rootNode = PlannerUtil.clone(masterPlan.getLogicalPlan().getRootBlock().getRoot());
+ planner.visitChild(masterPlan.getLogicalPlan(), rootNode, new Stack<LogicalNode>(), globalPlanContext);
+
+ ExecutionBlock terminalBlock = masterPlan.createTerminalBlock();
+
+ if (globalPlanContext.lastRepartionableNode != null
+ && globalPlanContext.lastRepartionableNode.getType() == NodeType.UNION) {
+ UnionNode unionNode = (UnionNode) globalPlanContext.lastRepartionableNode;
+ ConsecutiveUnionFinder finder = new ConsecutiveUnionFinder();
+ UnionsFinderContext finderContext = new UnionsFinderContext();
+ finder.visitChild(masterPlan.getLogicalPlan(), unionNode, new Stack<LogicalNode>(), finderContext);
+
+ for (UnionNode union : finderContext.unionList) {
+ TableSubQueryNode leftSubQuery = union.getLeftChild();
+ TableSubQueryNode rightSubQuery = union.getRightChild();
+ if (leftSubQuery.getSubQuery().getType() != NodeType.UNION) {
+ ExecutionBlock execBlock = masterPlan.newExecutionBlock();
+ execBlock.setPlan(leftSubQuery);
+ DataChannel dataChannel = new DataChannel(execBlock, terminalBlock, NONE_PARTITION, 1);
+ masterPlan.addConnect(dataChannel);
+ }
+ if (rightSubQuery.getSubQuery().getType() != NodeType.UNION) {
+ ExecutionBlock execBlock = masterPlan.newExecutionBlock();
+ execBlock.setPlan(rightSubQuery);
+ DataChannel dataChannel = new DataChannel(execBlock, terminalBlock, NONE_PARTITION, 1);
+ masterPlan.addConnect(dataChannel);
+ }
+ }
+ } else {
+ DataChannel dataChannel = new DataChannel(globalPlanContext.topMostLeftExecBlock, terminalBlock, NONE_PARTITION, 1);
+ dataChannel.setSchema(globalPlanContext.topmost.getOutSchema());
+ masterPlan.addConnect(dataChannel);
}
+ masterPlan.setTerminal(terminalBlock);
+ LOG.info(masterPlan);
+ }
- // insert store at the subnode of the root
- UnaryNode root = rootNode;
- if (root.getChild().getType() != NodeType.STORE) {
- ExecutionBlockId executionBlockId = QueryIdFactory.newExecutionBlockId(this.queryId);
- outputTableName = executionBlockId.toString();
- insertStore(executionBlockId.toString(),root).setLocal(false);
+ public static ScanNode buildInputExecutor(DataChannel channel) {
+ Preconditions.checkArgument(channel.getSchema() != null,
+ "Channel schema (" + channel.getSrcId().getId() +" -> "+ channel.getTargetId().getId()+") is not initialized");
+ TableMeta meta = new TableMetaImpl(channel.getSchema(), channel.getStoreType(), new Options());
+ TableDesc desc = new TableDescImpl(channel.getSrcId().toString(), meta, new Path("/"));
+ return new ScanNode(new FromTable(desc));
+ }
+
+ public class NewPlanner extends BasicLogicalPlanVisitor<GlobalPlanContext> {
+
+ @Override
+ public LogicalNode visitRoot(LogicalPlan plan, LogicalRootNode node, Stack<LogicalNode> stack,
+ GlobalPlanContext data) throws PlanningException {
+ super.visitRoot(plan, node, stack, data);
+
+ if (data.lastRepartionableNode != null && data.lastRepartionableNode.getType() != NodeType.UNION) {
+ data.topMostLeftExecBlock = addChannel(data.plan, data.lastRepartionableNode, node, data.topmost, data.topMostLeftExecBlock);
+ } else if (data.lastRepartionableNode != null && data.lastRepartionableNode.getType() == NodeType.UNION) {
+
+ } else {
+ ExecutionBlock execBlock = data.plan.newExecutionBlock();
+ execBlock.setPlan(node);
+ data.topMostLeftExecBlock = execBlock;
+ }
+
+ data.topmost = node;
+ return node;
}
-
- // convert 2-phase plan
- LogicalNode twoPhased = convertTo2Phase(rootNode);
- // make query graph
- MasterPlan globalPlan = convertToGlobalPlan(twoPhased);
- globalPlan.setOutputTableName(outputTableName);
+ @Override
+ public LogicalNode visitProjection(LogicalPlan plan, ProjectionNode node, Stack<LogicalNode> stack,
+ GlobalPlanContext data) throws PlanningException {
+ super.visitProjection(plan, node, stack, data);
+ data.topmost = node;
+ return node;
+ }
- return globalPlan;
- }
-
- private StoreTableNode insertStore(String tableId, LogicalNode parent) {
- StoreTableNode store = new StoreTableNode(tableId);
- store.setLocal(true);
- PlannerUtil.insertNode(parent, store);
- return store;
- }
-
- /**
- * Transforms a logical plan to a two-phase plan.
- * Store nodes are inserted for every logical nodes except store and scan nodes
- */
- private class GlobalPlanBuilder implements LogicalNodeVisitor {
@Override
- public void visit(LogicalNode node) {
- String tableId;
- StoreTableNode store;
- if (node.getType() == NodeType.GROUP_BY) {
- // transform group by to two-phase plan
- GroupbyNode groupby = (GroupbyNode) node;
- // insert a store for the child of first group by
- if (groupby.getChild().getType() != NodeType.UNION &&
- groupby.getChild().getType() != NodeType.STORE &&
- groupby.getChild().getType() != NodeType.SCAN) {
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- insertStore(tableId, groupby);
- }
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- // insert (a store for the first group by) and (a second group by)
- PlannerUtil.transformGroupbyTo2PWithStore((GroupbyNode)node, tableId);
- } else if (node.getType() == NodeType.SORT) {
- // transform sort to two-phase plan
- SortNode sort = (SortNode) node;
- // insert a store for the child of first sort
- if (sort.getChild().getType() != NodeType.UNION &&
- sort.getChild().getType() != NodeType.STORE &&
- sort.getChild().getType() != NodeType.SCAN) {
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- insertStore(tableId, sort);
- }
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- // insert (a store for the first sort) and (a second sort)
- PlannerUtil.transformSortTo2PWithStore((SortNode)node, tableId);
- } else if (node.getType() == NodeType.JOIN) {
- // transform join to two-phase plan
- // the first phase of two-phase join can be any logical nodes
- JoinNode join = (JoinNode) node;
-
- if (join.getRightChild().getType() == NodeType.SCAN &&
- join.getLeftChild().getType() == NodeType.SCAN) {
- ScanNode outerScan = (ScanNode) join.getRightChild();
- ScanNode innerScan = (ScanNode) join.getLeftChild();
-
-
- TableMeta outerMeta = outerScan.getFromTable().getTableDesc().getMeta();
- TableMeta innerMeta = innerScan.getFromTable().getTableDesc().getMeta();
- long threshold = conf.getLongVar(ConfVars.BROADCAST_JOIN_THRESHOLD);
-
-
- // if the broadcast join is available
- boolean outerSmall = false;
- boolean innerSmall = false;
- if (!outerScan.isLocal() && outerMeta.getStat() != null &&
- outerMeta.getStat().getNumBytes() <= threshold) {
- outerSmall = true;
- LOG.info("The relation (" + outerScan.getTableId() +
- ") is less than " + threshold);
+ public LogicalNode visitLimit(LogicalPlan plan, LimitNode node, Stack<LogicalNode> stack, GlobalPlanContext data) throws PlanningException {
+ super.visitLimit(plan, node, stack, data);
+ data.topmost = node;
+ return node;
+ }
+
+ private ExecutionBlock addChannel(MasterPlan masterPlan, LogicalNode lastDistNode, LogicalNode curNode,
+ LogicalNode childNode, ExecutionBlock lastChildBlock) throws PlanningException {
+ ExecutionBlock currentBlock = null;
+ ExecutionBlock childBlock;
+
+ childBlock = lastChildBlock;
+
+ NodeType shuffleRequiredNodeType = lastDistNode.getType();
+ if (shuffleRequiredNodeType == NodeType.GROUP_BY) {
+ GroupbyNode groupByNode = (GroupbyNode) lastDistNode;
+
+ GroupbyNode firstGroupBy = PlannerUtil.transformGroupbyTo2P(groupByNode);
+
+ if (firstGroupBy.getChild().getType() == NodeType.TABLE_SUBQUERY &&
+ ((TableSubQueryNode)firstGroupBy.getChild()).getSubQuery().getType() == NodeType.UNION) {
+
+ UnionNode unionNode = PlannerUtil.findTopNode(groupByNode, NodeType.UNION);
+ ConsecutiveUnionFinder finder = new ConsecutiveUnionFinder();
+ UnionsFinderContext finderContext = new UnionsFinderContext();
+ finder.visitChild(masterPlan.getLogicalPlan(), unionNode, new Stack<LogicalNode>(), finderContext);
+
+ currentBlock = masterPlan.newExecutionBlock();
+ GroupbyNode secondGroupBy = groupByNode;
+ for (UnionNode union : finderContext.unionList) {
+ TableSubQueryNode leftSubQuery = union.getLeftChild();
+ TableSubQueryNode rightSubQuery = union.getRightChild();
+ DataChannel dataChannel;
+ if (leftSubQuery.getSubQuery().getType() != NodeType.UNION) {
+ ExecutionBlock execBlock = masterPlan.newExecutionBlock();
+ GroupbyNode g1 = PlannerUtil.clone(firstGroupBy);
+ g1.setChild(leftSubQuery);
+ execBlock.setPlan(g1);
+ dataChannel = new DataChannel(execBlock, currentBlock, HASH_PARTITION, 32);
+
+ ScanNode scanNode = buildInputExecutor(dataChannel);
+ secondGroupBy.setChild(scanNode);
+ masterPlan.addConnect(dataChannel);
+ }
+ if (rightSubQuery.getSubQuery().getType() != NodeType.UNION) {
+ ExecutionBlock execBlock = masterPlan.newExecutionBlock();
+ GroupbyNode g1 = PlannerUtil.clone(firstGroupBy);
+ g1.setChild(rightSubQuery);
+ execBlock.setPlan(g1);
+ dataChannel = new DataChannel(execBlock, currentBlock, HASH_PARTITION, 32);
+
+ ScanNode scanNode = buildInputExecutor(dataChannel);
+ secondGroupBy.setChild(scanNode);
+ masterPlan.addConnect(dataChannel);
+ }
}
- if (!innerScan.isLocal() && innerMeta.getStat() != null &&
- innerMeta.getStat().getNumBytes() <= threshold) {
- innerSmall = true;
- LOG.info("The relation (" + innerScan.getTableId() +
- ") is less than " + threshold);
+ LogicalNode parent = PlannerUtil.findTopParentNode(curNode, lastDistNode.getType());
+ if (parent instanceof UnaryNode && parent != secondGroupBy) {
+ ((UnaryNode)parent).setChild(secondGroupBy);
}
+ currentBlock.setPlan(curNode);
+ } else {
- if (outerSmall && innerSmall) {
- if (outerMeta.getStat().getNumBytes() <=
- innerMeta.getStat().getNumBytes()) {
- outerScan.setBroadcast();
- LOG.info("The relation " + outerScan.getTableId()
- + " is broadcasted");
- } else {
- innerScan.setBroadcast();
- LOG.info("The relation " + innerScan.getTableId()
- + " is broadcasted");
- }
+ if (childBlock == null) { // first repartition node
+ childBlock = masterPlan.newExecutionBlock();
+ }
+ childBlock.setPlan(firstGroupBy);
+
+ currentBlock = masterPlan.newExecutionBlock();
+
+ DataChannel channel;
+ if (firstGroupBy.isEmptyGrouping()) {
+ channel = new DataChannel(childBlock, currentBlock, HASH_PARTITION, 1);
+ channel.setPartitionKey(firstGroupBy.getGroupingColumns());
} else {
- if (outerSmall) {
- outerScan.setBroadcast();
- LOG.info("The relation (" + outerScan.getTableId()
- + ") is broadcasted");
- } else if (innerSmall) {
- innerScan.setBroadcast();
- LOG.info("The relation (" + innerScan.getTableId()
- + ") is broadcasted");
- }
+ channel = new DataChannel(childBlock, currentBlock, HASH_PARTITION, 32);
+ channel.setPartitionKey(firstGroupBy.getGroupingColumns());
}
+ channel.setSchema(firstGroupBy.getOutSchema());
- if (outerScan.isBroadcast() || innerScan.isBroadcast()) {
- return;
+ GroupbyNode secondGroupBy = groupByNode;
+ ScanNode scanNode = buildInputExecutor(channel);
+ secondGroupBy.setChild(scanNode);
+
+ LogicalNode parent = PlannerUtil.findTopParentNode(curNode, lastDistNode.getType());
+ if (parent instanceof UnaryNode && parent != secondGroupBy) {
+ ((UnaryNode)parent).setChild(secondGroupBy);
}
- }
- // insert stores for the first phase
- if (join.getLeftChild().getType() != NodeType.UNION &&
- join.getLeftChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- store = new StoreTableNode(tableId);
- store.setLocal(true);
- PlannerUtil.insertOuterNode(node, store);
+ masterPlan.addConnect(channel);
+ currentBlock.setPlan(curNode);
}
- if (join.getRightChild().getType() != NodeType.UNION &&
- join.getRightChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- store = new StoreTableNode(tableId);
- store.setLocal(true);
- PlannerUtil.insertInnerNode(node, store);
+ } else if (shuffleRequiredNodeType == NodeType.SORT) {
+ SortNode firstSort = (SortNode) lastDistNode;
+ if (childBlock == null) {
+ childBlock = masterPlan.newExecutionBlock();
}
- } else if (node.getType() == NodeType.UNION) {
- // not two-phase transform
- UnionNode union = (UnionNode) node;
- // insert stores
- if (union.getLeftChild().getType() != NodeType.UNION &&
- union.getLeftChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- store = new StoreTableNode(tableId);
- if(union.getLeftChild().getType() == NodeType.GROUP_BY) {
- /*This case is for cube by operator
- * TODO : more complicated conidtion*/
- store.setLocal(true);
+ childBlock.setPlan(firstSort);
+
+ currentBlock = masterPlan.newExecutionBlock();
+ DataChannel channel = new DataChannel(childBlock, currentBlock, RANGE_PARTITION, 32);
+ channel.setPartitionKey(PlannerUtil.sortSpecsToSchema(firstSort.getSortKeys()).toArray());
+ channel.setSchema(childNode.getOutSchema());
+
+ SortNode secondSort = PlannerUtil.clone(lastDistNode);
+ ScanNode secondScan = buildInputExecutor(channel);
+ secondSort.setChild(secondScan);
+
+ LimitNode limitAndSort;
+ LimitNode limitOrNull = PlannerUtil.findTopNode(curNode, NodeType.LIMIT);
+ if (limitOrNull != null) {
+ limitAndSort = PlannerUtil.clone(limitOrNull);
+ limitAndSort.setChild(firstSort);
+
+ if (childBlock.getPlan().getType() == NodeType.SORT) {
+ childBlock.setPlan(limitAndSort);
} else {
- /* This case is for union query*/
- store.setLocal(false);
+ LogicalNode sortParent = PlannerUtil.findTopParentNode(childBlock.getPlan(), NodeType.SORT);
+ if (sortParent != null) {
+ if (sortParent instanceof UnaryNode) {
+ ((UnaryNode)sortParent).setChild(limitAndSort);
+ }
+ }
}
- PlannerUtil.insertOuterNode(node, store);
}
- if (union.getRightChild().getType() != NodeType.UNION &&
- union.getRightChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- store = new StoreTableNode(tableId);
- if(union.getRightChild().getType() == NodeType.GROUP_BY) {
- /*This case is for cube by operator
- * TODO : more complicated conidtion*/
- store.setLocal(true);
- }else {
- /* This case is for union query*/
- store.setLocal(false);
- }
- PlannerUtil.insertInnerNode(node, store);
+
+ LogicalNode parent = PlannerUtil.findTopParentNode(curNode, lastDistNode.getType());
+ if (parent instanceof UnaryNode && parent != secondSort) {
+ ((UnaryNode)parent).setChild(secondSort);
}
- } else if (node instanceof UnaryNode) {
- UnaryNode unary = (UnaryNode)node;
- if (unary.getType() != NodeType.STORE &&
- unary.getChild().getType() != NodeType.STORE) {
- tableId = QueryIdFactory.newExecutionBlockId(queryId).toString();
- insertStore(tableId, unary);
+
+ masterPlan.addConnect(channel);
+ currentBlock.setPlan(curNode);
+ } else if (shuffleRequiredNodeType == NodeType.JOIN) {
+ JoinNode joinNode = (JoinNode) lastDistNode;
+ LogicalNode leftNode = joinNode.getLeftChild();
+ LogicalNode rightNode = joinNode.getRightChild();
+
+ ExecutionBlock leftBlock = null;
+ if (lastChildBlock == null) {
+ leftBlock = masterPlan.newExecutionBlock();
+ leftBlock.setPlan(leftNode);
+ } else {
+ leftBlock = lastChildBlock;
}
+ ExecutionBlock rightBlock = masterPlan.newExecutionBlock();
+ rightBlock.setPlan(rightNode);
+
+ currentBlock = masterPlan.newExecutionBlock();
+
+ DataChannel leftChannel = new DataChannel(leftBlock, currentBlock, HASH_PARTITION, 32);
+ DataChannel rightChannel = new DataChannel(rightBlock, currentBlock, HASH_PARTITION, 32);
+
+ ScanNode leftScan = buildInputExecutor(leftChannel);
+ ScanNode rightScan = buildInputExecutor(rightChannel);
+
+ joinNode.setLeftChild(leftScan);
+ joinNode.setRightChild(rightScan);
+ currentBlock.setPlan(joinNode);
+
+ masterPlan.addConnect(leftChannel);
+ masterPlan.addConnect(rightChannel);
}
+
+ return currentBlock;
}
- }
- /**
- * Convert the logical plan to a two-phase plan by the post-order traverse.
- *
- * @param logicalPlan
- * @return
- */
- private LogicalNode convertTo2Phase(LogicalNode logicalPlan) {
- LogicalRootNode root = (LogicalRootNode) logicalPlan;
- root.postOrder(new GlobalPlanBuilder());
- return logicalPlan;
- }
-
- private Map<StoreTableNode, ExecutionBlock> convertMap =
- new HashMap<StoreTableNode, ExecutionBlock>();
-
- /**
- * Logical plan을 후위 탐색하면서 SubQuery 생성
- *
- * @param node 현재 방문 중인 노드
- * @throws IOException
- */
- private void recursiveBuildSubQuery(LogicalNode node)
- throws IOException {
- ExecutionBlock subQuery;
- StoreTableNode store;
- if (node instanceof UnaryNode) {
- recursiveBuildSubQuery(((UnaryNode) node).getChild());
-
- if (node.getType() == NodeType.STORE) {
- store = (StoreTableNode) node;
- ExecutionBlockId id;
- if (store.getTableName().startsWith(ExecutionBlockId.EB_ID_PREFIX)) {
- id = TajoIdUtils.createExecutionBlockId(store.getTableName());
- } else {
- id = QueryIdFactory.newExecutionBlockId(queryId);
- }
- subQuery = new ExecutionBlock(id);
-
- switch (store.getChild().getType()) {
- case BST_INDEX_SCAN:
- case SCAN: // store - scan
- subQuery = makeScanSubQuery(subQuery);
- subQuery.setPlan(node);
- break;
- case SELECTION:
- case PROJECTION:
- case LIMIT:
- subQuery = makeUnarySubQuery(store, node, subQuery);
- subQuery.setPlan(node);
- break;
- case GROUP_BY:
- subQuery = makeGroupbySubQuery(store, node, subQuery);
- subQuery.setPlan(node);
- break;
- case SORT:
- subQuery = makeSortSubQuery(store, node, subQuery);
- subQuery.setPlan(node);
- break;
- case JOIN: // store - join
- subQuery = makeJoinSubQuery(store, node, subQuery);
- subQuery.setPlan(node);
- break;
- case UNION:
- subQuery = makeUnionSubQuery(store, node, subQuery);
- subQuery.setPlan(node);
- break;
- default:
- subQuery = null;
- break;
- }
+ @Override
+ public LogicalNode visitSort(LogicalPlan plan, SortNode node, Stack<LogicalNode> stack, GlobalPlanContext data)
+ throws PlanningException {
- convertMap.put(store, subQuery);
+ super.visitSort(plan, node, stack, data);
+
+ if (data.lastRepartionableNode != null) {
+ data.topMostLeftExecBlock = addChannel(data.plan, data.lastRepartionableNode, node, data.topmost, data.topMostLeftExecBlock);
}
- } else if (node instanceof BinaryNode) {
- recursiveBuildSubQuery(((BinaryNode) node).getLeftChild());
- recursiveBuildSubQuery(((BinaryNode) node).getRightChild());
- } else if (node instanceof ScanNode) {
- } else {
+ data.topmost = node;
+ data.lastRepartionableNode = node;
- }
- }
-
- private ExecutionBlock makeScanSubQuery(ExecutionBlock block) {
- block.setPartitionType(PartitionType.LIST);
- return block;
- }
-
- /**
- * Unifiable node(selection, projection)을 자식 플랜과 같은 SubQuery로 생성
- *
- * @param rootStore 생성할 SubQuery의 store
- * @param plan logical plan
- * @param unit 생성할 SubQuery
- * @return
- * @throws IOException
- */
- private ExecutionBlock makeUnarySubQuery(StoreTableNode rootStore,
- LogicalNode plan, ExecutionBlock unit) throws IOException {
- ScanNode newScan;
- ExecutionBlock prev;
- UnaryNode unary = (UnaryNode) plan;
- UnaryNode child = (UnaryNode) unary.getChild();
- StoreTableNode prevStore = (StoreTableNode)child.getChild();
-
- // add scan
- newScan = GlobalPlannerUtils.newScanPlan(prevStore.getOutSchema(),
- prevStore.getTableName(), sm.getTablePath(prevStore.getTableName()));
- newScan.setLocal(true);
- child.setChild(newScan);
- prev = convertMap.get(prevStore);
-
- if (prev != null) {
- prev.setParentBlock(unit);
- unit.addChildBlock(newScan, prev);
- prev.setPartitionType(PartitionType.LIST);
+ return node;
}
- unit.setPartitionType(PartitionType.LIST);
+ @Override
+ public LogicalNode visitGroupBy(LogicalPlan plan, GroupbyNode node, Stack<LogicalNode> stack, GlobalPlanContext data) throws PlanningException {
+ super.visitGroupBy(plan, node, stack, data);
- return unit;
- }
-
- /**
- * Two-phase SubQuery 생성.
- *
- * @param rootStore 생성할 SubQuery의 store
- * @param plan logical plan
- * @param unit 생성할 SubQuery
- * @return
- * @throws IOException
- */
- private ExecutionBlock makeGroupbySubQuery(StoreTableNode rootStore,
- LogicalNode plan, ExecutionBlock unit) throws IOException {
- UnaryNode unary = (UnaryNode) plan;
- UnaryNode unaryChild;
- StoreTableNode prevStore;
- ScanNode newScan;
- ExecutionBlock prev;
- unaryChild = (UnaryNode) unary.getChild(); // groupby
- NodeType curType = unaryChild.getType();
- if (unaryChild.getChild().getType() == NodeType.STORE) {
- // store - groupby - store
- unaryChild = (UnaryNode) unaryChild.getChild(); // store
- prevStore = (StoreTableNode) unaryChild;
- newScan = GlobalPlannerUtils.newScanPlan(prevStore.getOutSchema(),
- prevStore.getTableName(),
- sm.getTablePath(prevStore.getTableName()));
- newScan.setLocal(true);
- ((UnaryNode) unary.getChild()).setChild(newScan);
- prev = convertMap.get(prevStore);
- if (prev != null) {
- prev.setParentBlock(unit);
- unit.addChildBlock(newScan, prev);
+ if (data.lastRepartionableNode != null) {
+ data.topMostLeftExecBlock = addChannel(data.plan, data.lastRepartionableNode, node, data.topmost, data.topMostLeftExecBlock);
}
- if (unaryChild.getChild().getType() == curType) {
- // the second phase
- unit.setPartitionType(PartitionType.LIST);
- if (prev != null) {
- prev.setPartitionType(PartitionType.HASH);
- }
- } else {
- // the first phase
- unit.setPartitionType(PartitionType.HASH);
- if (prev != null) {
- prev.setPartitionType(PartitionType.LIST);
- }
- }
- } else if (unaryChild.getChild().getType() == NodeType.SCAN) {
- // the first phase
- // store - groupby - scan
- unit.setPartitionType(PartitionType.HASH);
- } else if (unaryChild.getChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, (UnionNode)unaryChild.getChild(), unit,
- null, PartitionType.LIST);
- } else {
- // error
+ data.topmost = node;
+ data.lastRepartionableNode = node;
+ return node;
}
- return unit;
- }
-
- /**
- *
- *
- * @param rootStore 생성할 SubQuery의 store
- * @param plan logical plan
- * @param unit 생성할 SubQuery
- * @return
- * @throws IOException
- */
- private ExecutionBlock makeUnionSubQuery(StoreTableNode rootStore,
- LogicalNode plan, ExecutionBlock unit) throws IOException {
- UnaryNode unary = (UnaryNode) plan;
- StoreTableNode outerStore, innerStore;
- ExecutionBlock prev;
- UnionNode union = (UnionNode) unary.getChild();
- unit.setPartitionType(PartitionType.LIST);
-
- if (union.getLeftChild().getType() == NodeType.STORE) {
- outerStore = (StoreTableNode) union.getLeftChild();
- TableMeta outerMeta = CatalogUtil.newTableMeta(outerStore.getOutSchema(),
- StoreType.CSV);
- insertOuterScan(union, outerStore.getTableName(), outerMeta);
- prev = convertMap.get(outerStore);
- if (prev != null) {
- prev.getStoreTableNode().setTableName(rootStore.getTableName());
- prev.setPartitionType(PartitionType.LIST);
- prev.setParentBlock(unit);
- unit.addChildBlock((ScanNode) union.getLeftChild(), prev);
- }
- } else if (union.getLeftChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, union, unit, null, PartitionType.LIST);
+
+ @Override
+ public LogicalNode visitFilter(LogicalPlan plan, SelectionNode node, Stack<LogicalNode> stack, GlobalPlanContext data) throws PlanningException {
+ super.visitFilter(plan, node, stack, data);
+ data.topmost = node;
+ return node;
}
-
- if (union.getRightChild().getType() == NodeType.STORE) {
- innerStore = (StoreTableNode) union.getRightChild();
- TableMeta innerMeta = CatalogUtil.newTableMeta(innerStore.getOutSchema(),
- StoreType.CSV);
- insertInnerScan(union, innerStore.getTableName(), innerMeta);
- prev = convertMap.get(innerStore);
- if (prev != null) {
- prev.getStoreTableNode().setTableName(rootStore.getTableName());
- prev.setPartitionType(PartitionType.LIST);
- prev.setParentBlock(unit);
- unit.addChildBlock((ScanNode) union.getRightChild(), prev);
+
+ @Override
+ public LogicalNode visitJoin(LogicalPlan plan, JoinNode node, Stack<LogicalNode> stack, GlobalPlanContext data) throws PlanningException {
+ super.visitJoin(plan, node, stack, data);
+
+ if (data.lastRepartionableNode != null) {
+ data.topMostLeftExecBlock = addChannel(data.plan, data.lastRepartionableNode, node, data.topmost, data.topMostLeftExecBlock);
}
- } else if (union.getRightChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, union, unit, null, PartitionType.LIST);
+
+ data.topmost = node;
+ data.lastRepartionableNode = node;
+
+ return node;
}
- return unit;
- }
+ @Override
+ public LogicalNode visitUnion(LogicalPlan plan, UnionNode node, Stack<LogicalNode> stack, GlobalPlanContext data) throws PlanningException {
+ super.visitUnion(plan, node, stack, data);
- private ExecutionBlock makeSortSubQuery(StoreTableNode rootStore,
- LogicalNode plan, ExecutionBlock unit) throws IOException {
-
- UnaryNode unary = (UnaryNode) plan;
- UnaryNode unaryChild;
- StoreTableNode prevStore;
- ScanNode newScan;
- ExecutionBlock prev;
- unaryChild = (UnaryNode) unary.getChild(); // groupby
- NodeType curType = unaryChild.getType();
- if (unaryChild.getChild().getType() == NodeType.STORE) {
- // store - groupby - store
- unaryChild = (UnaryNode) unaryChild.getChild(); // store
- prevStore = (StoreTableNode) unaryChild;
- newScan = GlobalPlannerUtils.newScanPlan(prevStore.getOutSchema(),
- prevStore.getTableName(), sm.getTablePath(prevStore.getTableName()));
- newScan.setLocal(true);
- ((UnaryNode) unary.getChild()).setChild(newScan);
- prev = convertMap.get(prevStore);
- if (prev != null) {
- prev.setParentBlock(unit);
- unit.addChildBlock(newScan, prev);
- if (unaryChild.getChild().getType() == curType) {
- // TODO - this is duplicated code
- prev.setPartitionType(PartitionType.RANGE);
- } else {
- prev.setPartitionType(PartitionType.LIST);
- }
+ if (data.lastRepartionableNode != null && data.lastRepartionableNode.getType() != NodeType.UNION) {
+ data.topMostLeftExecBlock = addChannel(data.plan, data.lastRepartionableNode, node, data.topmost,
+ data.topMostLeftExecBlock);
}
- if (unaryChild.getChild().getType() == curType) {
- // the second phase
- unit.setPartitionType(PartitionType.LIST);
- } else {
- // the first phase
- unit.setPartitionType(PartitionType.HASH);
- }
- } else if (unaryChild.getChild().getType() == NodeType.SCAN) {
- // the first phase
- // store - sort - scan
- unit.setPartitionType(PartitionType.RANGE);
- } else if (unaryChild.getChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, (UnionNode)unaryChild.getChild(), unit,
- null, PartitionType.LIST);
- } else {
- // error
+
+ data.topmost = node;
+ data.lastRepartionableNode = node;
+ return node;
}
- return unit;
- }
-
- private ExecutionBlock makeJoinSubQuery(StoreTableNode rootStore,
- LogicalNode plan, ExecutionBlock unit) throws IOException {
- UnaryNode unary = (UnaryNode)plan;
- StoreTableNode outerStore, innerStore;
- ExecutionBlock prev;
- JoinNode join = (JoinNode) unary.getChild();
- Schema outerSchema = join.getLeftChild().getOutSchema();
- Schema innerSchema = join.getRightChild().getOutSchema();
- unit.setPartitionType(PartitionType.LIST);
-
- List<Column> outerCollist = new ArrayList<Column>();
- List<Column> innerCollist = new ArrayList<Column>();
-
- // TODO: set partition for store nodes
- if (join.hasJoinQual()) {
- // getting repartition keys
- List<Column[]> cols = PlannerUtil.getJoinKeyPairs(join.getJoinQual(), outerSchema, innerSchema);
- for (Column [] pair : cols) {
- outerCollist.add(pair[0]);
- innerCollist.add(pair[1]);
- }
- } else {
- // broadcast
+
+ @Override
+ public LogicalNode visitExcept(LogicalPlan plan, ExceptNode node, Stack<LogicalNode> stack,
+ GlobalPlanContext data) throws PlanningException {
+ super.visitExcept(plan, node, stack, data);
+ data.topmost = node;
+ return node;
}
-
- Column[] outerCols = new Column[outerCollist.size()];
- Column[] innerCols = new Column[innerCollist.size()];
- outerCols = outerCollist.toArray(outerCols);
- innerCols = innerCollist.toArray(innerCols);
-
- // outer
- if (join.getLeftChild().getType() == NodeType.STORE) {
- outerStore = (StoreTableNode) join.getLeftChild();
- TableMeta outerMeta = CatalogUtil.newTableMeta(outerStore.getOutSchema(),
- StoreType.CSV);
- insertOuterScan(join, outerStore.getTableName(), outerMeta);
- prev = convertMap.get(outerStore);
- if (prev != null) {
- prev.setPartitionType(PartitionType.HASH);
- prev.setParentBlock(unit);
- unit.addChildBlock((ScanNode) join.getLeftChild(), prev);
- }
- outerStore.setPartitions(PartitionType.HASH, outerCols, 32);
- } else if (join.getLeftChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, (UnionNode)join.getLeftChild(), unit,
- outerCols, PartitionType.HASH);
- } else {
+ @Override
+ public LogicalNode visitIntersect(LogicalPlan plan, IntersectNode node, Stack<LogicalNode> stack,
+ GlobalPlanContext data) throws PlanningException {
+ super.visitIntersect(plan, node, stack, data);
+ data.topmost = node;
+ return node;
}
-
- // inner
- if (join.getRightChild().getType() == NodeType.STORE) {
- innerStore = (StoreTableNode) join.getRightChild();
- TableMeta innerMeta = CatalogUtil.newTableMeta(innerStore.getOutSchema(),
- StoreType.CSV);
- insertInnerScan(join, innerStore.getTableName(), innerMeta);
- prev = convertMap.get(innerStore);
- if (prev != null) {
- prev.setPartitionType(PartitionType.HASH);
- prev.setParentBlock(unit);
- unit.addChildBlock((ScanNode) join.getRightChild(), prev);
- }
- innerStore.setPartitions(PartitionType.HASH, innerCols, 32);
- } else if (join.getRightChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, (UnionNode)join.getRightChild(), unit,
- innerCols, PartitionType.HASH);
+
+ @Override
+ public LogicalNode visitTableSubQuery(LogicalPlan plan, TableSubQueryNode node, Stack<LogicalNode> stack,
+ GlobalPlanContext data) throws PlanningException {
+ super.visitTableSubQuery(plan, node, stack, data);
+ data.topmost = node;
+ return node;
}
-
- return unit;
- }
-
- /**
- * Recursive하게 union의 자식 plan들을 설정
- *
- * @param rootStore 생성할 SubQuery의 store
- * @param union union을 root로 하는 logical plan
- * @param cur 생성할 SubQuery
- * @param cols partition 정보를 설정하기 위한 column array
- * @param prevOutputType 자식 SubQuery의 partition type
- * @throws IOException
- */
- private void _handleUnionNode(StoreTableNode rootStore, UnionNode union,
- ExecutionBlock cur, Column[] cols, PartitionType prevOutputType)
- throws IOException {
- StoreTableNode store;
- TableMeta meta;
- ExecutionBlock prev;
-
- if (union.getLeftChild().getType() == NodeType.STORE) {
- store = (StoreTableNode) union.getLeftChild();
- meta = CatalogUtil.newTableMeta(store.getOutSchema(), StoreType.CSV);
- insertOuterScan(union, store.getTableName(), meta);
- prev = convertMap.get(store);
- if (prev != null) {
- prev.getStoreTableNode().setTableName(rootStore.getTableName());
- prev.setPartitionType(prevOutputType);
- prev.setParentBlock(cur);
- cur.addChildBlock((ScanNode) union.getLeftChild(), prev);
- }
- if (cols != null) {
- store.setPartitions(PartitionType.LIST, cols, 32);
- }
- } else if (union.getLeftChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, (UnionNode)union.getLeftChild(), cur, cols,
- prevOutputType);
+
+ @Override
+ public LogicalNode visitScan(LogicalPlan plan, ScanNode node, Stack<LogicalNode> stack, GlobalPlanContext data)
+ throws PlanningException {
+ data.topmost = node;
+ return node;
}
-
- if (union.getRightChild().getType() == NodeType.STORE) {
- store = (StoreTableNode) union.getRightChild();
- meta = CatalogUtil.newTableMeta(store.getOutSchema(), StoreType.CSV);
- insertInnerScan(union, store.getTableName(), meta);
- prev = convertMap.get(store);
- if (prev != null) {
- prev.getStoreTableNode().setTableName(rootStore.getTableName());
- prev.setPartitionType(prevOutputType);
- prev.setParentBlock(cur);
- cur.addChildBlock((ScanNode) union.getRightChild(), prev);
- }
- if (cols != null) {
- store.setPartitions(PartitionType.LIST, cols, 32);
- }
- } else if (union.getRightChild().getType() == NodeType.UNION) {
- _handleUnionNode(rootStore, (UnionNode)union.getRightChild(), cur, cols,
- prevOutputType);
+
+ @Override
+ public LogicalNode visitStoreTable(LogicalPlan plan, StoreTableNode node, Stack<LogicalNode> stack,
+ GlobalPlanContext data) throws PlanningException {
+ super.visitStoreTable(plan, node, stack, data);
+ data.topmost = node;
+ return node;
+ }
+
+ @Override
+ public LogicalNode visitInsert(LogicalPlan plan, InsertNode node, Stack<LogicalNode> stack, GlobalPlanContext data)
+ throws PlanningException {
+ super.visitInsert(plan, node, stack, data);
+ data.topmost = node;
+ return node;
}
}
-
- private LogicalNode insertOuterScan(BinaryNode parent, String tableId,
- TableMeta meta) throws IOException {
- TableDesc desc = CatalogUtil.newTableDesc(tableId, meta, sm.getTablePath(tableId));
- ScanNode scan = new ScanNode(new FromTable(desc));
- scan.setLocal(true);
- scan.setInSchema(meta.getSchema());
- scan.setOutSchema(meta.getSchema());
- parent.setLeftChild(scan);
- return parent;
- }
-
- private LogicalNode insertInnerScan(BinaryNode parent, String tableId,
- TableMeta meta) throws IOException {
- TableDesc desc = CatalogUtil.newTableDesc(tableId, meta, sm.getTablePath(tableId));
- ScanNode scan = new ScanNode(new FromTable(desc));
- scan.setLocal(true);
- scan.setInSchema(meta.getSchema());
- scan.setOutSchema(meta.getSchema());
- parent.setRightChild(scan);
- return parent;
+
+ private class UnionsFinderContext {
+ List<UnionNode> unionList = new ArrayList<UnionNode>();
}
-
- private MasterPlan convertToGlobalPlan(LogicalNode logicalPlan) throws IOException {
- recursiveBuildSubQuery(logicalPlan);
- ExecutionBlock root;
- root = convertMap.get(((LogicalRootNode)logicalPlan).getChild());
- root.getStoreTableNode().setLocal(false);
+ private class ConsecutiveUnionFinder extends BasicLogicalPlanVisitor<UnionsFinderContext> {
+ public LogicalNode visitUnion(LogicalPlan plan, UnionNode node, Stack<LogicalNode> stack, UnionsFinderContext data)
+ throws PlanningException {
+ if (node.getType() == NodeType.UNION) {
+ data.unionList.add(node);
+ }
+
+ stack.push(node);
+ TableSubQueryNode leftSubQuery = node.getLeftChild();
+ TableSubQueryNode rightSubQuery = node.getRightChild();
+ if (leftSubQuery.getSubQuery().getType() == NodeType.UNION) {
+ visitChild(plan, leftSubQuery, stack, data);
+ }
+ if (rightSubQuery.getSubQuery().getType() == NodeType.UNION) {
+ visitChild(plan, rightSubQuery, stack, data);
+ }
+ stack.pop();
- return new MasterPlan(root);
+ return node;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryContext.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryContext.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryContext.java
index 8e89938..f921a15 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryContext.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/QueryContext.java
@@ -168,4 +168,12 @@ public class QueryContext extends Options {
public boolean isInsert() {
return getCommandType() == NodeType.INSERT;
}
+
+ public void setHiveQueryMode() {
+ setBool("hive.query.mode", true);
+ }
+
+ public boolean isHiveQueryMode() {
+ return getBool("hive.query.mode");
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
index 33a1e53..5875a6c 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TajoAsyncDispatcher.java
@@ -129,8 +129,6 @@ public class TajoAsyncDispatcher extends AbstractService implements Dispatcher
LOG.debug("Dispatching the event " + event.getClass().getName() + "."
+ event.toString());
}
-// LOG.info("====> Dispatching the event " + event.getClass().getName() + "."
-// + event.toString() );
Class<? extends Enum> type = event.getType().getDeclaringClass();
try{
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
index 574122b..24eea42 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/TaskSchedulerImpl.java
@@ -40,6 +40,7 @@ import org.apache.tajo.master.event.TaskSchedulerEvent;
import org.apache.tajo.master.event.TaskSchedulerEvent.EventType;
import org.apache.tajo.master.querymaster.QueryMasterTask;
import org.apache.tajo.master.querymaster.QueryUnit;
+import org.apache.tajo.master.querymaster.SubQuery;
import org.apache.tajo.storage.Fragment;
import org.apache.tajo.util.NetUtils;
@@ -454,17 +455,19 @@ public class TaskSchedulerImpl extends AbstractService
}
}
+ SubQuery subQuery = context.getQuery().getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId());
+
if (attemptId != null) {
- QueryUnit task = context.getQuery()
- .getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId()).getQueryUnit(attemptId.getQueryUnitId());
+ QueryUnit task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
attemptId,
new ArrayList<Fragment>(task.getAllFragments()),
- task.getOutputName(),
+ "",
false,
task.getLogicalPlan().toJson(),
- context.getQueryContext());
- if (task.getStoreTableNode().isLocal()) {
+ context.getQueryContext(),
+ subQuery.getDataChannel());
+ if (!subQuery.getBlock().isRoot()) {
taskAssign.setInterQuery();
}
@@ -500,23 +503,24 @@ public class TaskSchedulerImpl extends AbstractService
LOG.debug("Assigned based on * match");
QueryUnit task;
- task = context.getSubQuery(
- attemptId.getQueryUnitId().getExecutionBlockId()).getQueryUnit(attemptId.getQueryUnitId());
+ SubQuery subQuery = context.getSubQuery(attemptId.getQueryUnitId().getExecutionBlockId());
+ task = subQuery.getQueryUnit(attemptId.getQueryUnitId());
QueryUnitRequest taskAssign = new QueryUnitRequestImpl(
attemptId,
Lists.newArrayList(task.getAllFragments()),
- task.getOutputName(),
+ "",
false,
task.getLogicalPlan().toJson(),
- context.getQueryContext());
- if (task.getStoreTableNode().isLocal()) {
+ context.getQueryContext(),
+ subQuery.getDataChannel());
+ if (!subQuery.getBlock().isRoot()) {
taskAssign.setInterQuery();
}
for (ScanNode scan : task.getScanNodes()) {
Collection<URI> fetches = task.getFetch(scan);
if (fetches != null) {
for (URI fetch : fetches) {
- taskAssign.addFetch(scan.getTableId(), fetch);
+ taskAssign.addFetch(scan.getTableName(), fetch);
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
index 53c08c0..4618da6 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/event/QueryStartEvent.java
@@ -32,12 +32,14 @@ public class QueryStartEvent extends AbstractEvent {
private QueryId queryId;
private QueryContext queryContext;
+ private String sql;
private String logicalPlanJson;
- public QueryStartEvent(QueryId queryId, QueryContext queryContext, String logicalPlanJson) {
+ public QueryStartEvent(QueryId queryId, QueryContext queryContext, String sql, String logicalPlanJson) {
super(EventType.QUERY_START);
this.queryId = queryId;
this.queryContext = queryContext;
+ this.sql = sql;
this.logicalPlanJson = logicalPlanJson;
}
@@ -49,6 +51,10 @@ public class QueryStartEvent extends AbstractEvent {
return this.queryContext;
}
+ public String getSql() {
+ return this.sql;
+ }
+
public String getLogicalPlanJson() {
return logicalPlanJson;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
index 4ba95b0..4199e17 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/Query.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.state.*;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.ExecutionBlockId;
import org.apache.tajo.QueryId;
import org.apache.tajo.TajoConstants;
@@ -36,11 +37,11 @@ import org.apache.tajo.catalog.TableDesc;
import org.apache.tajo.catalog.TableDescImpl;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.engine.planner.global.MasterPlan;
+import org.apache.tajo.master.ExecutionBlock;
import org.apache.tajo.master.ExecutionBlockCursor;
import org.apache.tajo.master.QueryContext;
import org.apache.tajo.master.event.*;
import org.apache.tajo.storage.AbstractStorageManager;
-import org.apache.tajo.util.TUtil;
import java.io.IOException;
import java.util.*;
@@ -159,7 +160,7 @@ public class Query implements EventHandler<QueryEvent> {
}
float totalProgress = 0;
- float proportion = 1.0f / (float)getExecutionBlockCursor().size();
+ float proportion = 1.0f / (float)(getExecutionBlockCursor().size() - 1); // minus one is due to
for (int i = 0; i < subProgresses.length; i++) {
totalProgress += subProgresses[i] * proportion;
@@ -240,7 +241,7 @@ public class Query implements EventHandler<QueryEvent> {
}
public Collection<SubQuery> getSubQueries() {
- return Collections.unmodifiableCollection(this.subqueries.values());
+ return this.subqueries.values();
}
public QueryState getState() {
@@ -272,8 +273,8 @@ public class Query implements EventHandler<QueryEvent> {
@Override
public void transition(Query query, QueryEvent queryEvent) {
- SubQuery subQuery = new SubQuery(query.context, query.getExecutionBlockCursor().nextBlock(),
- query.sm);
+ SubQuery subQuery = new SubQuery(query.context, query.getPlan(),
+ query.getExecutionBlockCursor().nextBlock(), query.sm);
subQuery.setPriority(query.priority--);
query.addSubQuery(subQuery);
LOG.debug("Schedule unit plan: \n" + subQuery.getBlock().getPlan());
@@ -292,11 +293,12 @@ public class Query implements EventHandler<QueryEvent> {
query.completedSubQueryCount++;
SubQueryCompletedEvent castEvent = (SubQueryCompletedEvent) event;
ExecutionBlockCursor cursor = query.getExecutionBlockCursor();
-
+ MasterPlan masterPlan = query.getPlan();
// if the subquery is succeeded
if (castEvent.getFinalState() == SubQueryState.SUCCEEDED) {
- if (cursor.hasNext()) {
- SubQuery nextSubQuery = new SubQuery(query.context, cursor.nextBlock(), query.sm);
+ ExecutionBlock nextBlock = cursor.nextBlock();
+ if (!query.getPlan().isTerminal(nextBlock) || !query.getPlan().isRoot(nextBlock)) {
+ SubQuery nextSubQuery = new SubQuery(query.context, query.getPlan(), nextBlock, query.sm);
nextSubQuery.setPriority(query.priority--);
query.addSubQuery(nextSubQuery);
nextSubQuery.handle(new SubQueryEvent(nextSubQuery.getId(),
@@ -310,7 +312,7 @@ public class Query implements EventHandler<QueryEvent> {
} else { // Finish a query
if (query.checkQueryForCompleted() == QueryState.QUERY_SUCCEEDED) {
-
+ DataChannel finalChannel = masterPlan.getChannel(castEvent.getExecutionBlockId(), nextBlock.getId());
Path finalOutputDir = commitOutputData(query);
TableDesc finalTableDesc = buildOrUpdateResultTableDesc(query, castEvent.getExecutionBlockId(), finalOutputDir);
@@ -361,7 +363,8 @@ public class Query implements EventHandler<QueryEvent> {
/**
* It builds a table desc and update the table desc if necessary.
*/
- public TableDesc buildOrUpdateResultTableDesc(Query query, ExecutionBlockId finalExecBlockId, Path finalOutputDir) {
+ public TableDesc buildOrUpdateResultTableDesc(Query query, ExecutionBlockId finalExecBlockId,
+ Path finalOutputDir) {
// Determine the output table name
SubQuery subQuery = query.getSubQuery(finalExecBlockId);
QueryContext queryContext = query.context.getQueryContext();
@@ -450,33 +453,4 @@ public class Query implements EventHandler<QueryEvent> {
writeLock.unlock();
}
}
-
- public static interface QueryHook {
- QueryState getTargetState();
- void onEvent(Query query);
- }
-
- public static class QueryHookManager {
- Map<QueryState, List<QueryHook>> hookList = TUtil.newHashMap();
-
- public void addHook(QueryHook hook) {
- if (hookList.containsKey(hook.getTargetState())) {
- hookList.get(hook.getTargetState()).add(hook);
- } else {
- hookList.put(hook.getTargetState(), TUtil.newList(hook));
- }
- }
-
- public void doHooks(Query query) {
- QueryState finalState = query.checkQueryForCompleted();
- List<QueryHook> list = hookList.get(finalState);
- if (list != null) {
- for (QueryHook hook : list) {
- hook.onEvent(query);
- }
- } else {
- LOG.error("QueryHookManager cannot deal with " + finalState + " event");
- }
- }
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
index c54f8da..53dfb6a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryInProgress.java
@@ -207,6 +207,7 @@ public class QueryInProgress extends CompositeService {
TajoWorkerProtocol.QueryExecutionRequestProto.newBuilder()
.setQueryId(queryId.getProto())
.setQueryContext(queryContext.getProto())
+ .setSql(PrimitiveProtos.StringProto.newBuilder().setValue(queryInfo.getSql()))
.setLogicalPlanJson(PrimitiveProtos.StringProto.newBuilder().setValue(plan.toJson()).build())
.build(), NullCallback.get());
querySubmitted.set(true);
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
index 6611102..29dcb1d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.service.Service;
import org.apache.tajo.QueryId;
import org.apache.tajo.TajoProtos;
import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.planner.global.GlobalOptimizer;
+import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.ipc.TajoMasterProtocol;
import org.apache.tajo.master.GlobalPlanner;
import org.apache.tajo.master.TajoAsyncDispatcher;
@@ -55,8 +55,6 @@ public class QueryMaster extends CompositeService implements EventHandler {
private GlobalPlanner globalPlanner;
- private GlobalOptimizer globalOptimizer;
-
private AbstractStorageManager storageManager;
private TajoConf systemConf;
@@ -93,8 +91,7 @@ public class QueryMaster extends CompositeService implements EventHandler {
this.storageManager = StorageManagerFactory.getStorageManager(systemConf);
- globalPlanner = new GlobalPlanner(systemConf, storageManager, dispatcher.getEventHandler());
- globalOptimizer = new GlobalOptimizer();
+ globalPlanner = new GlobalPlanner(systemConf, storageManager);
dispatcher.register(QueryStartEvent.EventType.class, new QueryStartEventHandler());
@@ -217,9 +214,6 @@ public class QueryMaster extends CompositeService implements EventHandler {
public GlobalPlanner getGlobalPlanner() {
return globalPlanner;
}
- public GlobalOptimizer getGlobalOptimizer() {
- return globalOptimizer;
- }
public TajoWorker.WorkerContext getWorkerContext() {
return workerContext;
@@ -253,9 +247,8 @@ public class QueryMaster extends CompositeService implements EventHandler {
@Override
public void handle(QueryStartEvent event) {
LOG.info("Start QueryStartEventHandler:" + event.getQueryId());
- //To change body of implemented methods use File | Settings | File Templates.
QueryMasterTask queryMasterTask = new QueryMasterTask(queryMasterContext,
- event.getQueryId(), event.getQueryContext(), event.getLogicalPlanJson());
+ event.getQueryId(), event.getQueryContext(), event.getSql(), event.getLogicalPlanJson());
queryMasterTask.init(systemConf);
queryMasterTask.start();
@@ -280,17 +273,21 @@ public class QueryMaster extends CompositeService implements EventHandler {
}
synchronized(queryMasterTasks) {
for(QueryMasterTask eachTask: tempTasks) {
- TajoMasterProtocol.TajoHeartbeat queryHeartbeat = TajoMasterProtocol.TajoHeartbeat.newBuilder()
- .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
- .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
- .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
- .setState(eachTask.getState())
- .setQueryId(eachTask.getQueryId().getProto())
- .setQueryProgress(eachTask.getQuery().getProgress())
- .setQueryFinishTime(eachTask.getQuery().getFinishTime())
- .build();
-
- workerContext.getTajoMasterRpcClient().heartbeat(null, queryHeartbeat, NullCallback.get());
+ try {
+ TajoMasterProtocol.TajoHeartbeat queryHeartbeat = TajoMasterProtocol.TajoHeartbeat.newBuilder()
+ .setTajoWorkerHost(workerContext.getTajoWorkerManagerService().getBindAddr().getHostName())
+ .setTajoWorkerPort(workerContext.getTajoWorkerManagerService().getBindAddr().getPort())
+ .setTajoWorkerClientPort(workerContext.getTajoWorkerClientService().getBindAddr().getPort())
+ .setState(eachTask.getState())
+ .setQueryId(eachTask.getQueryId().getProto())
+ .setQueryProgress(eachTask.getQuery().getProgress())
+ .setQueryFinishTime(eachTask.getQuery().getFinishTime())
+ .build();
+
+ workerContext.getTajoMasterRpcClient().heartbeat(null, queryHeartbeat, NullCallback.get());
+ } catch (Throwable t) {
+ t.printStackTrace();
+ }
}
}
synchronized(queryMasterStop) {
@@ -309,7 +306,7 @@ public class QueryMaster extends CompositeService implements EventHandler {
class ClientSessionTimeoutCheckThread extends Thread {
public void run() {
LOG.info("ClientSessionTimeoutCheckThread started");
- while(true) {
+ while(!queryMasterStop.get()) {
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
@@ -337,5 +334,4 @@ public class QueryMaster extends CompositeService implements EventHandler {
}
}
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
index a4fabcf..e760626 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java
@@ -30,16 +30,20 @@ import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.tajo.*;
+import org.apache.tajo.algebra.Expr;
+import org.apache.tajo.catalog.CatalogService;
import org.apache.tajo.catalog.TableDesc;
import org.apache.tajo.conf.TajoConf;
-import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.engine.planner.PlannerUtil;
+import org.apache.tajo.engine.parser.HiveConverter;
+import org.apache.tajo.engine.parser.SQLAnalyzer;
+import org.apache.tajo.engine.planner.*;
import org.apache.tajo.engine.planner.global.MasterPlan;
import org.apache.tajo.engine.planner.logical.LogicalNode;
-import org.apache.tajo.engine.planner.logical.LogicalRootNode;
import org.apache.tajo.engine.planner.logical.NodeType;
import org.apache.tajo.engine.planner.logical.ScanNode;
import org.apache.tajo.master.QueryContext;
+import org.apache.tajo.master.GlobalEngine;
+import org.apache.tajo.master.QueryContext;
import org.apache.tajo.master.TajoAsyncDispatcher;
import org.apache.tajo.master.event.*;
import org.apache.tajo.master.rm.TajoWorkerResourceManager;
@@ -74,6 +78,8 @@ public class QueryMasterTask extends CompositeService {
private MasterPlan masterPlan;
+ private String sql;
+
private String logicalPlanJson;
private TajoAsyncDispatcher dispatcher;
@@ -91,11 +97,12 @@ public class QueryMasterTask extends CompositeService {
private AtomicBoolean stopped = new AtomicBoolean(false);
public QueryMasterTask(QueryMaster.QueryMasterContext queryMasterContext,
- QueryId queryId, QueryContext queryContext, String logicalPlanJson) {
+ QueryId queryId, QueryContext queryContext, String sql, String logicalPlanJson) {
super(QueryMasterTask.class.getName());
this.queryMasterContext = queryMasterContext;
this.queryId = queryId;
this.queryContext = queryContext;
+ this.sql = sql;
this.logicalPlanJson = logicalPlanJson;
this.querySubmitTime = System.currentTimeMillis();
}
@@ -227,17 +234,44 @@ public class QueryMasterTask extends CompositeService {
return;
}
+ CatalogService catalog = getQueryTaskContext().getQueryMasterContext().getWorkerContext().getCatalog();
+ LogicalPlanner planner = new LogicalPlanner(catalog);
+ LogicalOptimizer optimizer = new LogicalOptimizer();
+ Expr expr;
+ if (queryContext.isHiveQueryMode()) {
+ HiveConverter hiveConverter = new HiveConverter();
+ expr = hiveConverter.parse(sql);
+ } else {
+ SQLAnalyzer analyzer = new SQLAnalyzer();
+ expr = analyzer.parse(sql);
+ }
+ LogicalPlan plan = null;
try {
- LogicalRootNode logicalNodeRoot = (LogicalRootNode) CoreGsonHelper.fromJson(logicalPlanJson, LogicalNode.class);
- LogicalNode[] scanNodes = PlannerUtil.findAllNodes(logicalNodeRoot, NodeType.SCAN);
- if(scanNodes != null) {
- for(LogicalNode eachScanNode: scanNodes) {
- ScanNode scanNode = (ScanNode)eachScanNode;
- tableDescMap.put(scanNode.getFromTable().getTableName(), scanNode.getFromTable().getTableDesc());
+ plan = planner.createPlan(expr);
+ optimizer.optimize(plan);
+ } catch (PlanningException e) {
+ e.printStackTrace();
+ }
+
+ GlobalEngine.DistributedQueryHookManager hookManager = new GlobalEngine.DistributedQueryHookManager();
+ hookManager.addHook(new GlobalEngine.InsertHook());
+ hookManager.doHooks(queryContext, plan);
+
+ try {
+
+ for (LogicalPlan.QueryBlock block : plan.getQueryBlocks()) {
+ LogicalNode[] scanNodes = PlannerUtil.findAllNodes(block.getRoot(), NodeType.SCAN);
+ if(scanNodes != null) {
+ for(LogicalNode eachScanNode: scanNodes) {
+ ScanNode scanNode = (ScanNode)eachScanNode;
+ tableDescMap.put(scanNode.getFromTable().getTableName(), scanNode.getFromTable().getTableDesc());
+ }
}
}
- MasterPlan globalPlan = queryMasterContext.getGlobalPlanner().build(queryId, logicalNodeRoot);
- this.masterPlan = queryMasterContext.getGlobalOptimizer().optimize(globalPlan);
+
+ MasterPlan masterPlan = new MasterPlan(queryId, queryContext, plan);
+ queryMasterContext.getGlobalPlanner().build(masterPlan);
+ //this.masterPlan = queryMasterContext.getGlobalOptimizer().optimize(masterPlan);
query = new Query(queryTaskContext, queryId, querySubmitTime,
"", queryTaskContext.getEventHandler(), masterPlan);
@@ -306,9 +340,9 @@ public class QueryMasterTask extends CompositeService {
LOG.info("The staging dir '" + outputDir + "' is created.");
queryContext.setStagingDir(stagingDir);
- ////////////////////////////////////////////////////
- // Check and Create An Output Directory If Necessary
- ////////////////////////////////////////////////////
+ /////////////////////////////////////////////////
+ // Check and Create Output Directory If Necessary
+ /////////////////////////////////////////////////
if (queryContext.hasOutputPath()) {
outputDir = queryContext.getOutputPath();
if (queryContext.isOutputOverwrite()) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
index 00dcc0b..5a30c04 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/querymaster/QueryUnit.java
@@ -156,10 +156,7 @@ public class QueryUnit implements EventHandler<TaskEvent> {
}
public void setLogicalPlan(LogicalNode plan) {
- Preconditions.checkArgument(plan.getType() == NodeType.STORE);
-
this.plan = plan;
- store = (StoreTableNode) plan;
LogicalNode node = plan;
ArrayList<LogicalNode> s = new ArrayList<LogicalNode>();
@@ -250,7 +247,7 @@ public class QueryUnit implements EventHandler<TaskEvent> {
}
public Collection<URI> getFetch(ScanNode scan) {
- return this.fetchMap.get(scan.getTableId());
+ return this.fetchMap.get(scan.getTableName());
}
public String getOutputName() {
[6/7] TAJO-184: Refactor GlobalPlanner and global plan data
structure. (hyunsik)
Posted by hy...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
index e9ed087..351b6fc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java
@@ -45,6 +45,7 @@ import java.util.List;
import java.util.Stack;
import static org.apache.tajo.algebra.Aggregation.GroupType;
+import static org.apache.tajo.engine.planner.LogicalPlan.BlockType;
/**
* This class creates a logical plan from a parse tree ({@link org.apache.tajo.engine.parser.SQLAnalyzer})
@@ -135,6 +136,16 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
}
}
+ public TableSubQueryNode visitTableSubQuery(PlanContext context, Stack<OpType> stack, TableSubQuery expr) throws PlanningException {
+ QueryBlock newBlock = context.plan.newAndGetBlock(expr.getName());
+ PlanContext newContext = new PlanContext(context.plan, newBlock);
+ Stack<OpType> newStack = new Stack<OpType>();
+ LogicalNode child = visitChild(newContext, newStack, expr.getSubQuery());
+ context.plan.connectBlocks(newContext.block, context.block, BlockType.TableSubQuery);
+ return new TableSubQueryNode(expr.getName(), child);
+ }
+
+
@Override
public ScanNode visitRelation(PlanContext context, Stack<OpType> stack, Relation expr)
throws VerifyException {
@@ -307,22 +318,26 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
PlanContext leftContext = new PlanContext(plan, plan.newAnonymousBlock());
Stack<OpType> leftStack = new Stack<OpType>();
LogicalNode left = visitChild(leftContext, leftStack, setOperation.getLeft());
+ TableSubQueryNode leftSubQuery = new TableSubQueryNode(leftContext.block.getName(), left);
+ context.plan.connectBlocks(leftContext.block, context.block, BlockType.TableSubQuery);
PlanContext rightContext = new PlanContext(plan, plan.newAnonymousBlock());
Stack<OpType> rightStack = new Stack<OpType>();
LogicalNode right = visitChild(rightContext, rightStack, setOperation.getRight());
+ TableSubQueryNode rightSubQuery = new TableSubQueryNode(rightContext.block.getName(), right);
+ context.plan.connectBlocks(rightContext.block, context.block, BlockType.TableSubQuery);
verifySetStatement(setOperation.getType(), leftContext.block, rightContext.block);
BinaryNode setOp;
if (setOperation.getType() == OpType.Union) {
- setOp = new UnionNode(left, right);
+ setOp = new UnionNode(leftSubQuery, rightSubQuery);
} else if (setOperation.getType() == OpType.Except) {
- setOp = new ExceptNode(left, right);
+ setOp = new ExceptNode(leftSubQuery, rightSubQuery);
} else if (setOperation.getType() == OpType.Intersect) {
- setOp = new IntersectNode(left, right);
+ setOp = new IntersectNode(leftSubQuery, rightSubQuery);
} else {
- throw new VerifyException(setOperation.toJson());
+ throw new VerifyException("Invalid Type: " + setOperation.getType());
}
// Strip the table names from the targets of the both blocks
@@ -330,10 +345,8 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
Target [] leftStrippedTargets = PlannerUtil.stripTarget(leftContext.block.getCurrentTargets());
Schema outSchema = PlannerUtil.targetToSchema(leftStrippedTargets);
- setOp.setInSchema(left.getOutSchema());
+ setOp.setInSchema(leftSubQuery.getOutSchema());
setOp.setOutSchema(outSchema);
- setOp.setLeftChild(left);
- setOp.setRightChild(right);
if (isNoUpperProjection(stack)) {
block.targetListManager = new TargetListManager(plan, leftStrippedTargets);
@@ -357,7 +370,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
for (int i = 0; i < targets1.length; i++) {
if (!targets1[i].getDataType().equals(targets2[i].getDataType())) {
- throw new VerifyException("UNION types " + targets1[i].getDataType().getType() + " and "
+ throw new VerifyException(type + " types " + targets1[i].getDataType().getType() + " and "
+ targets2[i].getDataType().getType() + " cannot be matched");
}
}
@@ -428,7 +441,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
// 4. Set Child Plan and Update Input Schemes Phase
groupingNode.setChild(child);
block.setGroupingNode(groupingNode);
- groupingNode.setInSchema(child.getInSchema());
+ groupingNode.setInSchema(child.getOutSchema());
// 5. Update Output Schema and Targets for Upper Plan
@@ -438,7 +451,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
List<Column[]> cuboids = generateCuboids(annotateGroupingColumn(plan, block.getName(),
groupElements[0].getColumns(), child));
UnionNode topUnion = createGroupByUnion(plan, block, child, cuboids, 0);
- block.resolveGrouping();
+ block.needToResolveGrouping();
block.getTargetListManager().setEvaluatedAll();
return topUnion;
@@ -827,7 +840,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
PlanContext newContext = new PlanContext(ctx.plan, newQueryBlock);
Stack<OpType> subStack = new Stack<OpType>();
LogicalNode subQuery = visitChild(newContext, subStack, expr.getSubQuery());
- ctx.plan.connectBlocks(ctx.block, newQueryBlock, QueryBlockGraph.BlockType.TableSubQuery);
+ ctx.plan.connectBlocks(newQueryBlock, ctx.block, BlockType.TableSubQuery);
stack.pop();
InsertNode insertNode = null;
@@ -849,7 +862,7 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
}
}
- checkInsertDomains(targetSchema, subQuery.getOutSchema());
+ ensureDomains(targetSchema, subQuery.getOutSchema());
insertNode = new InsertNode(desc, subQuery);
insertNode.setTargetSchema(targetSchema);
insertNode.setOutSchema(targetSchema);
@@ -872,12 +885,15 @@ public class LogicalPlanner extends BaseAlgebraVisitor<LogicalPlanner.PlanContex
return insertNode;
}
- private static void checkInsertDomains(Schema targetTableScheme, Schema insertSchema)
+ /**
+ * This ensures that corresponding columns in both tables are equivalent to each other.
+ */
+ private static void ensureDomains(Schema targetTableScheme, Schema schema)
throws PlanningException {
- for (int i = 0; i < insertSchema.getColumnNum(); i++) {
- if (!insertSchema.getColumn(i).getDataType().equals(targetTableScheme.getColumn(i).getDataType())) {
+ for (int i = 0; i < schema.getColumnNum(); i++) {
+ if (!schema.getColumn(i).getDataType().equals(targetTableScheme.getColumn(i).getDataType())) {
Column targetColumn = targetTableScheme.getColumn(i);
- Column insertColumn = insertSchema.getColumn(i);
+ Column insertColumn = schema.getColumn(i);
throw new PlanningException("ERROR: " +
insertColumn.getColumnName() + " is of type " + insertColumn.getDataType().getType().name() +
", but target column '" + targetColumn.getColumnName() + "' is of type " +
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
index 7d68e19..365731b 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PhysicalPlannerImpl.java
@@ -25,9 +25,11 @@ import com.google.common.base.Preconditions;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
+import org.apache.tajo.DataChannel;
import org.apache.tajo.TaskAttemptContext;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.catalog.SortSpec;
+import org.apache.tajo.catalog.proto.CatalogProtos;
import org.apache.tajo.conf.TajoConf;
import org.apache.tajo.engine.planner.logical.*;
import org.apache.tajo.engine.planner.physical.*;
@@ -39,6 +41,8 @@ import org.apache.tajo.util.IndexUtil;
import java.io.IOException;
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType;
+
public class PhysicalPlannerImpl implements PhysicalPlanner {
private static final Log LOG = LogFactory.getLog(PhysicalPlannerImpl.class);
protected final TajoConf conf;
@@ -49,19 +53,42 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
this.sm = sm;
}
- public PhysicalExec createPlan(final TaskAttemptContext context,
- final LogicalNode logicalPlan) throws InternalException {
+ public PhysicalExec createPlan(final TaskAttemptContext context, final LogicalNode logicalPlan)
+ throws InternalException {
- PhysicalExec plan;
+ PhysicalExec execPlan;
try {
- plan = createPlanRecursive(context, logicalPlan);
-
+ execPlan = createPlanRecursive(context, logicalPlan);
+ if (execPlan instanceof StoreTableExec || execPlan instanceof IndexedStoreExec
+ || execPlan instanceof PartitionedStoreExec) {
+ return execPlan;
+ } else if (context.getDataChannel() != null) {
+ return buildOutputOperator(context, logicalPlan, execPlan);
+ } else {
+ return execPlan;
+ }
} catch (IOException ioe) {
throw new InternalException(ioe);
}
+ }
- return plan;
+ private PhysicalExec buildOutputOperator(TaskAttemptContext context, LogicalNode plan,
+ PhysicalExec execPlan) throws IOException {
+ DataChannel channel = context.getDataChannel();
+ StoreTableNode storeTableNode = new StoreTableNode(channel.getTargetId().toString());
+ storeTableNode.setStorageType(CatalogProtos.StoreType.CSV);
+ storeTableNode.setInSchema(execPlan.getSchema());
+ storeTableNode.setOutSchema(execPlan.getSchema());
+ if (channel.getPartitionType() != PartitionType.NONE_PARTITION) {
+ storeTableNode.setPartitions(channel.getPartitionType(), channel.getPartitionKey(), channel.getPartitionNum());
+ } else {
+ storeTableNode.setDefaultParition();
+ }
+ storeTableNode.setChild(plan);
+
+ PhysicalExec outExecPlan = createStorePlan(context, storeTableNode, execPlan);
+ return outExecPlan;
}
private PhysicalExec createPlanRecursive(TaskAttemptContext ctx, LogicalNode logicalNode) throws IOException {
@@ -93,7 +120,12 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
outer = createPlanRecursive(ctx, prjNode.getChild());
return new ProjectionExec(ctx, prjNode, outer);
- case SCAN:
+ case TABLE_SUBQUERY: {
+ TableSubQueryNode subQueryNode = (TableSubQueryNode) logicalNode;
+ outer = createPlanRecursive(ctx, subQueryNode.getSubQuery());
+ return outer;
+
+ } case SCAN:
outer = createScanPlan(ctx, (ScanNode) logicalNode);
return outer;
@@ -202,17 +234,20 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
public PhysicalExec createStorePlan(TaskAttemptContext ctx,
StoreTableNode plan, PhysicalExec subOp) throws IOException {
- if (plan.hasPartitionKey()) {
- switch (plan.getPartitionType()) {
- case HASH:
+ if (plan.getPartitionType() == PartitionType.HASH_PARTITION
+ || plan.getPartitionType() == PartitionType.RANGE_PARTITION) {
+ switch (ctx.getDataChannel().getPartitionType()) {
+ case HASH_PARTITION:
return new PartitionedStoreExec(ctx, sm, plan, subOp);
- case RANGE:
+ case RANGE_PARTITION:
+ SortExec sortExec = PhysicalPlanUtil.findExecutor(subOp, SortExec.class);
+
SortSpec [] sortSpecs = null;
- if (subOp instanceof SortExec) {
- sortSpecs = ((SortExec)subOp).getSortSpecs();
+ if (sortExec != null) {
+ sortSpecs = sortExec.getSortSpecs();
} else {
- Column[] columns = plan.getPartitionKeys();
+ Column[] columns = ctx.getDataChannel().getPartitionKey();
SortSpec specs[] = new SortSpec[columns.length];
for (int i = 0; i < columns.length; i++) {
specs[i] = new SortSpec(columns[i]);
@@ -232,10 +267,10 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
public PhysicalExec createScanPlan(TaskAttemptContext ctx, ScanNode scanNode)
throws IOException {
- Preconditions.checkNotNull(ctx.getTable(scanNode.getTableId()),
- "Error: There is no table matched to %s", scanNode.getTableId());
+ Preconditions.checkNotNull(ctx.getTable(scanNode.getTableName()),
+ "Error: There is no table matched to %s", scanNode.getTableName());
- Fragment[] fragments = ctx.getTables(scanNode.getTableId());
+ Fragment[] fragments = ctx.getTables(scanNode.getTableName());
return new SeqScanExec(ctx, sm, scanNode, fragments);
}
@@ -281,14 +316,14 @@ public class PhysicalPlannerImpl implements PhysicalPlanner {
IndexScanNode annotation)
throws IOException {
//TODO-general Type Index
- Preconditions.checkNotNull(ctx.getTable(annotation.getTableId()),
- "Error: There is no table matched to %s", annotation.getTableId());
+ Preconditions.checkNotNull(ctx.getTable(annotation.getTableName()),
+ "Error: There is no table matched to %s", annotation.getTableName());
- Fragment[] fragments = ctx.getTables(annotation.getTableId());
+ Fragment[] fragments = ctx.getTables(annotation.getTableName());
String indexName = IndexUtil.getIndexNameOfFrag(fragments[0],
annotation.getSortKeys());
- Path indexPath = new Path(sm.getTablePath(annotation.getTableId()), "index");
+ Path indexPath = new Path(sm.getTablePath(annotation.getTableName()), "index");
TupleComparator comp = new TupleComparator(annotation.getKeySchema(),
annotation.getSortKeys());
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlanString.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlanString.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlanString.java
new file mode 100644
index 0000000..ef8bed0
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlanString.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+public class PlanString {
+ final StringBuilder title;
+
+ final List<String> explanations = new ArrayList<String>();
+ final List<String> details = new ArrayList<String>();
+
+ StringBuilder currentExplanation;
+ StringBuilder currentDetail;
+
+ public PlanString(String title) {
+ this.title = new StringBuilder(title);
+ }
+
+ public PlanString appendTitle(String str) {
+ title.append(str);
+ return this;
+ }
+
+ public PlanString addExplan(String explain) {
+ flushCurrentExplanation();
+ currentExplanation = new StringBuilder(explain);
+ return this;
+ }
+
+ public PlanString appendExplain(String explain) {
+ if (currentExplanation == null) {
+ currentExplanation = new StringBuilder();
+ }
+ currentExplanation.append(explain);
+ return this;
+ }
+
+ public PlanString addDetail(String detail) {
+ flushCurrentDetail();
+ currentDetail = new StringBuilder(detail);
+ return this;
+ }
+
+ public PlanString appendDetail(String detail) {
+ if (currentDetail == null) {
+ currentDetail = new StringBuilder();
+ }
+ currentDetail.append(detail);
+ return this;
+
+ }
+
+ public String getTitle() {
+ return title.toString();
+ }
+
+ public List<String> getExplanations() {
+ flushCurrentExplanation();
+ return explanations;
+ }
+
+ public List<String> getDetails() {
+ flushCurrentDetail();
+ return details;
+ }
+
+ private void flushCurrentExplanation() {
+ if (currentExplanation != null) {
+ explanations.add(currentExplanation.toString());
+ currentExplanation = null;
+ }
+ }
+
+ private void flushCurrentDetail() {
+ if (currentDetail != null) {
+ details.add(currentDetail.toString());
+ currentDetail = null;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java
index 3f686af..a06b07e 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java
@@ -61,7 +61,7 @@ public class PlannerUtil {
ScanNode scan;
for (int i = 0; i < scans.length; i++) {
scan = (ScanNode) scans[i];
- tableNames[i] = scan.getTableId();
+ tableNames[i] = scan.getTableName();
}
return tableNames;
}
@@ -127,65 +127,16 @@ public class PlannerUtil {
parentNode.setChild(newNode);
}
- public static LogicalNode insertOuterNode(LogicalNode parent, LogicalNode outer) {
- Preconditions.checkArgument(parent instanceof BinaryNode);
- Preconditions.checkArgument(outer instanceof UnaryNode);
-
- BinaryNode p = (BinaryNode) parent;
- LogicalNode c = p.getLeftChild();
- UnaryNode m = (UnaryNode) outer;
- m.setInSchema(c.getOutSchema());
- m.setOutSchema(c.getOutSchema());
- m.setChild(c);
- p.setLeftChild(m);
- return p;
- }
-
- public static LogicalNode insertInnerNode(LogicalNode parent, LogicalNode inner) {
- Preconditions.checkArgument(parent instanceof BinaryNode);
- Preconditions.checkArgument(inner instanceof UnaryNode);
-
- BinaryNode p = (BinaryNode) parent;
- LogicalNode c = p.getRightChild();
- UnaryNode m = (UnaryNode) inner;
- m.setInSchema(c.getOutSchema());
- m.setOutSchema(c.getOutSchema());
- m.setChild(c);
- p.setRightChild(m);
- return p;
- }
-
- public static LogicalNode insertNode(LogicalNode parent,
- LogicalNode left, LogicalNode right) {
- Preconditions.checkArgument(parent instanceof BinaryNode);
- Preconditions.checkArgument(left instanceof UnaryNode);
- Preconditions.checkArgument(right instanceof UnaryNode);
-
- BinaryNode p = (BinaryNode)parent;
- LogicalNode lc = p.getLeftChild();
- LogicalNode rc = p.getRightChild();
- UnaryNode lm = (UnaryNode)left;
- UnaryNode rm = (UnaryNode)right;
- lm.setInSchema(lc.getOutSchema());
- lm.setOutSchema(lc.getOutSchema());
- lm.setChild(lc);
- rm.setInSchema(rc.getOutSchema());
- rm.setOutSchema(rc.getOutSchema());
- rm.setChild(rc);
- p.setLeftChild(lm);
- p.setRightChild(rm);
- return p;
- }
-
- public static LogicalNode transformGroupbyTo2P(GroupbyNode gp) {
- Preconditions.checkNotNull(gp);
-
+ public static GroupbyNode transformGroupbyTo2P(GroupbyNode groupBy) {
+ Preconditions.checkNotNull(groupBy);
+
+ GroupbyNode child = null;
try {
// cloning groupby node
- GroupbyNode child = (GroupbyNode) gp.clone();
+ child = (GroupbyNode) groupBy.clone();
List<Target> newChildTargets = Lists.newArrayList();
- Target[] secondTargets = gp.getTargets();
+ Target[] secondTargets = groupBy.getTargets();
Target[] firstTargets = child.getTargets();
Target second;
@@ -231,45 +182,17 @@ public class PlannerUtil {
child.setTargets(targetArray);
child.setOutSchema(PlannerUtil.targetToSchema(targetArray));
// set the groupby chaining
- gp.setChild(child);
- gp.setInSchema(child.getOutSchema());
+ groupBy.setChild(child);
+ groupBy.setInSchema(child.getOutSchema());
} catch (CloneNotSupportedException e) {
LOG.error(e);
}
- return gp;
- }
-
- public static LogicalNode transformSortTo2P(SortNode sort) {
- Preconditions.checkNotNull(sort);
-
- try {
- SortNode child = (SortNode) sort.clone();
- sort.setChild(child);
- sort.setInSchema(child.getOutSchema());
- sort.setOutSchema(child.getOutSchema());
- } catch (CloneNotSupportedException e) {
- LOG.error(e);
- }
- return sort;
- }
-
- public static LogicalNode transformGroupbyTo2PWithStore(GroupbyNode gb,
- String tableId) {
- GroupbyNode groupby = (GroupbyNode) transformGroupbyTo2P(gb);
- return insertStore(groupby, tableId);
- }
-
- public static LogicalNode transformSortTo2PWithStore(SortNode sort,
- String tableId) {
- SortNode sort2p = (SortNode) transformSortTo2P(sort);
- return insertStore(sort2p, tableId);
+ return child;
}
- private static LogicalNode insertStore(LogicalNode parent,
- String tableId) {
- StoreTableNode store = new StoreTableNode(tableId);
- store.setLocal(true);
+ private static LogicalNode insertStore(LogicalNode parent, String tableName) {
+ StoreTableNode store = new StoreTableNode(tableName);
insertNode(parent, store);
return parent;
@@ -282,7 +205,7 @@ public class PlannerUtil {
* @param type to find
* @return a found logical node
*/
- public static LogicalNode findTopNode(LogicalNode node, NodeType type) {
+ public static <T extends LogicalNode> T findTopNode(LogicalNode node, NodeType type) {
Preconditions.checkNotNull(node);
Preconditions.checkNotNull(type);
@@ -292,7 +215,7 @@ public class PlannerUtil {
if (finder.getFoundNodes().size() == 0) {
return null;
}
- return finder.getFoundNodes().get(0);
+ return (T) finder.getFoundNodes().get(0);
}
/**
@@ -344,8 +267,8 @@ public class PlannerUtil {
Set<String> tableIds = Sets.newHashSet();
// getting distinct table references
for (Column col : columnRefs) {
- if (!tableIds.contains(col.getTableName())) {
- tableIds.add(col.getTableName());
+ if (!tableIds.contains(col.getQualifier())) {
+ tableIds.add(col.getQualifier());
}
}
@@ -373,11 +296,12 @@ public class PlannerUtil {
return i.contains(it.next()) && o.contains(it.next());
} else {
- if (node instanceof ScanNode) {
- ScanNode scan = (ScanNode) node;
+ if (node instanceof RelationNode) {
+
+ RelationNode scan = (RelationNode) node;
for (Column col : columnRefs) {
- if (scan.getTableId().equals(col.getTableName())) {
+ if (scan.getTableName().equals(col.getQualifier())) {
Column found = node.getInSchema().getColumnByName(col.getColumnName());
if (found == null) {
return false;
@@ -461,12 +385,6 @@ public class PlannerUtil {
}
}
- public static Set<Column> collectColumnRefs(LogicalNode node) {
- ColumnRefCollector collector = new ColumnRefCollector();
- node.postOrder(collector);
- return collector.getColumns();
- }
-
private static class ColumnRefCollector implements LogicalNodeVisitor {
private Set<Column> collected = Sets.newHashSet();
@@ -609,7 +527,7 @@ public class PlannerUtil {
List<Column> right = EvalTreeUtil.findAllColumnRefs(qual.getRightExpr());
if (left.size() == 1 && right.size() == 1 &&
- !left.get(0).getTableName().equals(right.get(0).getTableName()))
+ !left.get(0).getQualifier().equals(right.get(0).getQualifier()))
return true;
}
@@ -717,7 +635,7 @@ public class PlannerUtil {
}
if (copy[i].getEvalTree().getType() == EvalType.FIELD) {
FieldEval fieldEval = (FieldEval) copy[i].getEvalTree();
- if (fieldEval.getColumnRef().isQualified()) {
+ if (fieldEval.getColumnRef().hasQualifier()) {
fieldEval.getColumnRef().setName(fieldEval.getColumnName());
}
}
@@ -725,4 +643,19 @@ public class PlannerUtil {
return copy;
}
+
+ public static <T extends LogicalNode> T clone(LogicalNode node) {
+ try {
+ return (T) node.clone();
+ } catch (CloneNotSupportedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static Schema getQualifiedSchema(Schema targetSchema, String qualifier) {
+ Schema copied;
+ copied = (Schema) targetSchema.clone();
+ copied.setQualifier(qualifier);
+ return copied;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Projector.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Projector.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Projector.java
index 543bbd7..12e4978 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Projector.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Projector.java
@@ -65,14 +65,14 @@ public class Projector {
continue outer;
}
- col = inSchema.getColumn(outSchema.getColumn(targetId).getQualifiedName());
+ col = inSchema.getColumnByFQN(outSchema.getColumn(targetId).getQualifiedName());
outMap[mapId] = targetId;
inMap[mapId] = inSchema.getColumnId(col.getQualifiedName());
mapId++;
}
} else {
for (int targetId = 0; targetId < outSchema.getColumnNum(); targetId ++) {
- col = inSchema.getColumn(outSchema.getColumn(targetId).getQualifiedName());
+ col = inSchema.getColumnByFQN(outSchema.getColumn(targetId).getQualifiedName());
outMap[mapId] = targetId;
inMap[mapId] = inSchema.getColumnId(col.getQualifiedName());
mapId++;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/QueryBlockGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/QueryBlockGraph.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/QueryBlockGraph.java
deleted file mode 100644
index 8677ba8..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/QueryBlockGraph.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.planner;
-
-import org.apache.tajo.util.TUtil;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-
-/**
- * It provides a graph data structure for query blocks and their relations.
- */
-public class QueryBlockGraph {
- private Map<String, List<BlockEdge>> blockEdges = new HashMap<String, List<BlockEdge>>();
-
- public QueryBlockGraph() {
- }
-
- public int size() {
- return blockEdges.size();
- }
-
- public void connectBlocks(String srcBlock, String targetBlock, BlockType type) {
- BlockEdge newBlockEdge = new BlockEdge(targetBlock, type);
- if (blockEdges.containsKey(srcBlock)) {
- blockEdges.get(srcBlock).add(newBlockEdge);
- } else {
- blockEdges.put(srcBlock, TUtil.newList(newBlockEdge));
- }
- }
-
- public Collection<BlockEdge> getBlockEdges(String srcBlock) {
- return blockEdges.get(srcBlock);
- }
-
- public String toString() {
- StringBuilder sb = new StringBuilder();
- for (Map.Entry<String, List<BlockEdge>> entry : blockEdges.entrySet()) {
- sb.append(entry.getKey()).append("\n");
- for (BlockEdge edge : entry.getValue()) {
- sb.append(" |- ");
- sb.append(edge.getTargetBlock()).append(" (").append(edge.getBlockType()).append(")");
- }
- sb.append("\n");
- }
-
- return sb.toString();
- }
-
- public static enum BlockType {
- TableSubQuery,
- ScalarSubQuery
- }
-
- public static class BlockEdge {
- private String targetBlock;
- private BlockType blockType;
-
- public BlockEdge(String targetBlock, BlockType blockType) {
- this.targetBlock = targetBlock;
- this.blockType = blockType;
- }
-
- public String getTargetBlock() {
- return targetBlock;
- }
-
- public BlockType getBlockType() {
- return blockType;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
index c14fe88..670cbfb 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/Target.java
@@ -38,8 +38,7 @@ public class Target implements Cloneable, GsonObject {
public Target(EvalNode expr) {
this.expr = expr;
- if (expr.getType() == EvalType.AGG_FUNCTION &&
- expr.getValueType().length > 1) { // hack for partial result
+ if (expr.getType() == EvalType.AGG_FUNCTION && expr.getValueType().length > 1) { // hack for partial result
this.column = new Column(expr.getName(), Type.ARRAY);
} else {
this.column = new Column(expr.getName(), expr.getValueType()[0]);
@@ -48,11 +47,16 @@ public class Target implements Cloneable, GsonObject {
public Target(final EvalNode eval, final String alias) {
this(eval);
- this.alias = alias;
+ setAlias(alias);
+ }
+
+ public String getCanonicalName() {
+ return !hasAlias() ? column.getQualifiedName() : alias;
}
public final void setAlias(String alias) {
this.alias = alias;
+ this.column = new Column(alias, expr.getValueType()[0]);
}
public final String getAlias() {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/GlobalOptimizer.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/GlobalOptimizer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/GlobalOptimizer.java
deleted file mode 100644
index 36190ef..0000000
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/GlobalOptimizer.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.engine.planner.global;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.tajo.engine.planner.PlannerUtil;
-import org.apache.tajo.engine.planner.logical.LogicalNode;
-import org.apache.tajo.engine.planner.logical.NodeType;
-import org.apache.tajo.engine.planner.logical.ScanNode;
-import org.apache.tajo.engine.planner.logical.UnaryNode;
-import org.apache.tajo.master.ExecutionBlock;
-import org.apache.tajo.master.ExecutionBlock.PartitionType;
-
-public class GlobalOptimizer {
-
- public GlobalOptimizer() {
-
- }
-
- public MasterPlan optimize(MasterPlan plan) {
- ExecutionBlock reducedStep = reduceSchedules(plan.getRoot());
-
- MasterPlan optimized = new MasterPlan(reducedStep);
- optimized.setOutputTableName(plan.getOutputTable());
-
- return optimized;
- }
-
- @VisibleForTesting
- private ExecutionBlock reduceSchedules(ExecutionBlock logicalUnit) {
- reduceLogicalQueryUnitStep_(logicalUnit);
- return logicalUnit;
- }
-
- private void reduceLogicalQueryUnitStep_(ExecutionBlock cur) {
- if (cur.hasChildBlock()) {
- for (ExecutionBlock childBlock: cur.getChildBlocks())
- reduceLogicalQueryUnitStep_(childBlock);
- }
-
- for (ExecutionBlock childBlock: cur.getChildBlocks()) {
- if (childBlock.getStoreTableNode().getChild().getType() != NodeType.UNION &&
- childBlock.getPartitionType() == PartitionType.LIST) {
- mergeLogicalUnits(cur, childBlock);
- }
- }
- }
-
- private ExecutionBlock mergeLogicalUnits(ExecutionBlock parent, ExecutionBlock child) {
- LogicalNode p = PlannerUtil.findTopParentNode(parent.getPlan(), NodeType.SCAN);
-
- if (p instanceof UnaryNode) {
- UnaryNode u = (UnaryNode) p;
- ScanNode scan = (ScanNode) u.getChild();
- LogicalNode c = child.getStoreTableNode().getChild();
-
- parent.removeChildBlock(scan);
- u.setChild(c);
- parent.setPlan(parent.getPlan());
- parent.addChildBlocks(child.getChildBlockMap());
- }
- return parent;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java
index 9201997..7b6caa9 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/global/MasterPlan.java
@@ -21,17 +21,71 @@
*/
package org.apache.tajo.engine.planner.global;
+import org.apache.tajo.DataChannel;
+import org.apache.tajo.ExecutionBlockId;
+import org.apache.tajo.QueryId;
+import org.apache.tajo.engine.planner.LogicalPlan;
+import org.apache.tajo.engine.planner.graph.SimpleDirectedGraph;
import org.apache.tajo.master.ExecutionBlock;
+import org.apache.tajo.master.ExecutionBlockCursor;
+import org.apache.tajo.master.QueryContext;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.tajo.ipc.TajoWorkerProtocol.PartitionType;
public class MasterPlan {
+ private final QueryId queryId;
+ private final QueryContext context;
+ private final LogicalPlan plan;
private ExecutionBlock root;
- private String outputTableName;
+ private AtomicInteger nextId = new AtomicInteger(0);
+
+ private ExecutionBlock terminalBlock;
+ private Map<ExecutionBlockId, ExecutionBlock> execBlockMap = new HashMap<ExecutionBlockId, ExecutionBlock>();
+ private SimpleDirectedGraph<ExecutionBlockId, DataChannel> execBlockGraph =
+ new SimpleDirectedGraph<ExecutionBlockId, DataChannel>();
+
+ public ExecutionBlockId newExecutionBlockId() {
+ return new ExecutionBlockId(queryId, nextId.incrementAndGet());
+ }
+
+ public boolean isTerminal(ExecutionBlock execBlock) {
+ return terminalBlock == execBlock;
+ }
+
+ public ExecutionBlock getTerminalBlock() {
+ return terminalBlock;
+ }
+
+ public ExecutionBlock createTerminalBlock() {
+ terminalBlock = newExecutionBlock();
+ return terminalBlock;
+ }
+
+ public MasterPlan(QueryId queryId, QueryContext context, LogicalPlan plan) {
+ this.queryId = queryId;
+ this.context = context;
+ this.plan = plan;
+ }
+
+ public QueryId getQueryId() {
+ return this.queryId;
+ }
+
+ public QueryContext getContext() {
+ return this.context;
+ }
- public MasterPlan(ExecutionBlock root) {
- setRoot(root);
+ public LogicalPlan getLogicalPlan() {
+ return this.plan;
}
- public void setRoot(ExecutionBlock root) {
+ public void setTerminal(ExecutionBlock root) {
this.root = root;
}
@@ -39,11 +93,158 @@ public class MasterPlan {
return this.root;
}
- public void setOutputTableName(String tableName) {
- this.outputTableName = tableName;
+ public ExecutionBlock newExecutionBlock() {
+ ExecutionBlock newExecBlock = new ExecutionBlock(newExecutionBlockId());
+ execBlockMap.put(newExecBlock.getId(), newExecBlock);
+ return newExecBlock;
+ }
+
+ public boolean containsExecBlock(ExecutionBlockId execBlockId) {
+ return execBlockMap.containsKey(execBlockId);
+ }
+
+ public ExecutionBlock getExecBlock(ExecutionBlockId execBlockId) {
+ return execBlockMap.get(execBlockId);
+ }
+
+ public void addConnect(DataChannel dataChannel) {
+ execBlockGraph.connect(dataChannel.getSrcId(), dataChannel.getTargetId(), dataChannel);
+ }
+
+ public void addConnect(ExecutionBlock src, ExecutionBlock target, PartitionType type) {
+ addConnect(src.getId(), target.getId(), type);
+ }
+
+ public void addConnect(ExecutionBlockId src, ExecutionBlockId target, PartitionType type) {
+ addConnect(new DataChannel(src, target, type));
+ }
+
+ public boolean isConnected(ExecutionBlock src, ExecutionBlock target) {
+ return isConnected(src.getId(), target.getId());
+ }
+
+ public boolean isConnected(ExecutionBlockId src, ExecutionBlockId target) {
+ return execBlockGraph.isConnected(src, target);
+ }
+
+ public boolean isReverseConnected(ExecutionBlock target, ExecutionBlock src) {
+ return execBlockGraph.isReversedConnected(target.getId(), src.getId());
+ }
+
+ public boolean isReverseConnected(ExecutionBlockId target, ExecutionBlockId src) {
+ return execBlockGraph.isReversedConnected(target, src);
+ }
+
+ public DataChannel getChannel(ExecutionBlock src, ExecutionBlock target) {
+ return execBlockGraph.getEdge(src.getId(), target.getId());
+ }
+
+ public DataChannel getChannel(ExecutionBlockId src, ExecutionBlockId target) {
+ return execBlockGraph.getEdge(src, target);
+ }
+
+ public List<DataChannel> getOutgoingChannels(ExecutionBlockId src) {
+ return execBlockGraph.getOutgoingEdges(src);
+ }
+
+ public boolean isRoot(ExecutionBlock execBlock) {
+ return execBlockGraph.isRoot(execBlock.getId());
+ }
+
+ public boolean isLeaf(ExecutionBlock execBlock) {
+ return execBlockGraph.isLeaf(execBlock.getId());
+ }
+
+ public boolean isLeaf(ExecutionBlockId id) {
+ return execBlockGraph.isLeaf(id);
+ }
+
+ public List<DataChannel> getIncomingChannels(ExecutionBlockId target) {
+ return execBlockGraph.getIncomingEdges(target);
+ }
+
+ public void disconnect(ExecutionBlock src, ExecutionBlock target) {
+ disconnect(src.getId(), target.getId());
}
- public String getOutputTable() {
- return outputTableName;
+ public void disconnect(ExecutionBlockId src, ExecutionBlockId target) {
+ execBlockGraph.disconnect(src, target);
}
+
+ public List<ExecutionBlock> getChilds(ExecutionBlock execBlock) {
+ return getChilds(execBlock.getId());
+ }
+
+ public List<ExecutionBlock> getChilds(ExecutionBlockId id) {
+ List<ExecutionBlock> childBlocks = new ArrayList<ExecutionBlock>();
+ for (ExecutionBlockId cid : execBlockGraph.getChilds(id)) {
+ childBlocks.add(execBlockMap.get(cid));
+ }
+ return childBlocks;
+ }
+
+ public int getChildCount(ExecutionBlockId blockId) {
+ return execBlockGraph.getChildCount(blockId);
+ }
+
+ public ExecutionBlock getChild(ExecutionBlockId execBlockId, int idx) {
+ return execBlockMap.get(execBlockGraph.getChild(execBlockId, idx));
+ }
+
+ public ExecutionBlock getChild(ExecutionBlock executionBlock, int idx) {
+ return getChild(executionBlock.getId(), idx);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ ExecutionBlockCursor cursor = new ExecutionBlockCursor(this);
+ sb.append("-------------------------------------------------------------------------------\n");
+ sb.append("Execution Block Graph (TERMINAL - " + getTerminalBlock() + ")\n");
+ sb.append("-------------------------------------------------------------------------------\n");
+ sb.append(execBlockGraph.toStringGraph(getRoot().getId()));
+ sb.append("-------------------------------------------------------------------------------\n");
+
+ while(cursor.hasNext()) {
+ ExecutionBlock block = cursor.nextBlock();
+
+ boolean terminal = false;
+ sb.append("\n");
+ sb.append("=======================================================\n");
+ sb.append("Block Id: " + block.getId());
+ if (isTerminal(block)) {
+ sb.append(" [TERMINAL]");
+ terminal = true;
+ } else if (isRoot(block)) {
+ sb.append(" [ROOT]");
+ } else if (isLeaf(block)) {
+ sb.append(" [LEAF]");
+ } else {
+ sb.append(" [INTERMEDIATE]");
+ }
+ sb.append("\n");
+ sb.append("=======================================================\n");
+ if (terminal) {
+ continue;
+ }
+
+ if (!isLeaf(block)) {
+ sb.append("\n[Incoming]\n");
+ for (DataChannel channel : getIncomingChannels(block.getId())) {
+ sb.append(channel).append("\n");
+ }
+ }
+ sb.append("\n[Outgoing]\n");
+ if (!isRoot(block)) {
+ for (DataChannel channel : getOutgoingChannels(block.getId())) {
+ sb.append(channel);
+ sb.append("\n");
+ }
+ }
+ sb.append("\n").append(block.getPlan());
+ }
+
+ return sb.toString();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java
new file mode 100644
index 0000000..df428bb
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraph.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.graph;
+
+import java.util.List;
+
+/**
+ * This represents a directed graph.
+ *
+ * @param <V> The vertex class type
+ * @param <E> The edge class type
+ */
+public interface DirectedGraph<V, E> {
+
+ int size();
+
+ void connect(V tail, V head, E edge);
+
+ void disconnect(V tail, V head);
+
+ boolean isConnected(V tail, V head);
+
+ boolean isReversedConnected(V head, V tail);
+
+ E getEdge(V tail, V head);
+
+ E getReverseEdge(V head, V tail);
+
+ int getChildCount(V v);
+
+ List<E> getIncomingEdges(V head);
+
+ List<E> getOutgoingEdges(V tail);
+
+ List<V> getChilds(V v);
+
+ V getChild(V block, int idx);
+
+ V getParent(V v);
+
+ boolean isRoot(V v);
+
+ boolean isLeaf(V v);
+
+ /**
+ * It visits all vertices in a post-order traverse way.
+ */
+ void accept(V src, DirectedGraphVisitor<V> visitor);
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java
new file mode 100644
index 0000000..3deec7f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/DirectedGraphVisitor.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.graph;
+
+import java.util.Stack;
+
+public interface DirectedGraphVisitor<V> {
+ void visit(Stack<V> stack, V v);
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java
new file mode 100644
index 0000000..3bc360f
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/graph/SimpleDirectedGraph.java
@@ -0,0 +1,219 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.graph;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.tajo.util.TUtil;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Stack;
+
+/**
+ * This represents a simple directed graph. It does not support multiple edges between both vertices.
+ *
+ * @param <V> The vertex class type
+ * @param <E> The edge class type
+ */
+public class SimpleDirectedGraph<V, E> implements DirectedGraph<V,E> {
+ /** map: child -> parent */
+ private Map<V, Map<V, E>> directedEdges = TUtil.newLinkedHashMap();
+ /** map: parent -> child */
+ private Map<V, Map<V, E>> reversedEdges = TUtil.newLinkedHashMap();
+
+ @Override
+ public int size() {
+ return directedEdges.size();
+ }
+
+ @Override
+ public void connect(V tail, V head, E edge) {
+ TUtil.putToNestedMap(directedEdges, tail, head, edge);
+ TUtil.putToNestedMap(reversedEdges, head, tail, edge);
+ }
+
+ @Override
+ public void disconnect(V tail, V head) {
+ if (directedEdges.containsKey(tail)) {
+ directedEdges.get(tail).remove(head);
+ if (directedEdges.get(tail).isEmpty()) {
+ directedEdges.remove(tail);
+ }
+
+ reversedEdges.get(head).remove(tail);
+ if (reversedEdges.get(head).isEmpty()) {
+ reversedEdges.remove(head);
+ }
+ } else {
+ throw new RuntimeException("Not connected channel: " + tail + " -> " + head);
+ }
+ }
+
+ @Override
+ public boolean isConnected(V tail, V head) {
+ return directedEdges.containsKey(tail) && directedEdges.get(tail).containsKey(head);
+ }
+
+ @Override
+ public boolean isReversedConnected(V head, V tail) {
+ return reversedEdges.containsKey(head) && reversedEdges.get(head).containsKey(tail);
+ }
+
+ @Override
+ public E getEdge(V tail, V head) {
+ if (isConnected(tail, head)) {
+ return directedEdges.get(tail).get(head);
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public E getReverseEdge(V head, V tail) {
+ if (isReversedConnected(head, tail)) {
+ return reversedEdges.get(head).get(tail);
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public int getChildCount(V v) {
+ if (reversedEdges.containsKey(v)) {
+ return reversedEdges.get(v).size();
+ } else {
+ return -1;
+ }
+ }
+
+ @Override
+ public List<E> getIncomingEdges(V head) {
+ if (reversedEdges.containsKey(head)) {
+ return ImmutableList.copyOf(reversedEdges.get(head).values());
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public List<E> getOutgoingEdges(V tail) {
+ if (directedEdges.containsKey(tail)) {
+ return ImmutableList.copyOf(directedEdges.get(tail).values());
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public List<V> getChilds(V v) {
+ List<V> childBlocks = new ArrayList<V>();
+ if (reversedEdges.containsKey(v)) {
+ for (Map.Entry<V, E> entry: reversedEdges.get(v).entrySet()) {
+ childBlocks.add(entry.getKey());
+ }
+ }
+ return childBlocks;
+ }
+
+ @Override
+ public V getChild(V block, int idx) {
+ return getChilds(block).get(idx);
+ }
+
+ @Override
+ public V getParent(V v) {
+ if (directedEdges.containsKey(v)) {
+ return directedEdges.get(v).keySet().iterator().next();
+ } else {
+ return null;
+ }
+ }
+
+ @Override
+ public boolean isRoot(V v) {
+ return !directedEdges.containsKey(v);
+ }
+
+ @Override
+ public boolean isLeaf(V v) {
+ return !reversedEdges.containsKey(v);
+ }
+
+ @Override
+ public void accept(V source, DirectedGraphVisitor<V> visitor) {
+ Stack<V> stack = new Stack<V>();
+ visitRecursive(stack, source, visitor);
+ }
+
+ private void visitRecursive(Stack<V> stack, V current, DirectedGraphVisitor<V> visitor) {
+ stack.push(current);
+ for (V child : getChilds(current)) {
+ visitRecursive(stack, child, visitor);
+ }
+ stack.pop();
+ visitor.visit(stack, current);
+ }
+
+ public String toString() {
+ return "G (|v| = " + directedEdges.size() +")";
+ }
+
+ public String printDepthString(DepthString planStr) {
+ StringBuilder output = new StringBuilder();
+ String pad = new String(new char[planStr.depth * 3]).replace('\0', ' ');
+ output.append(pad + "|-" + planStr.vertexStr).append("\n");
+
+ return output.toString();
+ }
+
+ public String toStringGraph(V vertex) {
+ StringBuilder sb = new StringBuilder();
+ QueryGraphTopologyStringBuilder visitor = new QueryGraphTopologyStringBuilder();
+ accept(vertex, visitor);
+ Stack<DepthString> depthStrings = visitor.getDepthStrings();
+ while(!depthStrings.isEmpty()) {
+ sb.append(printDepthString(depthStrings.pop()));
+ }
+ return sb.toString();
+ }
+
+ private class DepthString {
+ int depth;
+ String vertexStr;
+
+ DepthString(int depth, String vertexStr) {
+ this.depth = depth;
+ this.vertexStr = vertexStr;
+ }
+ }
+
+ private class QueryGraphTopologyStringBuilder implements DirectedGraphVisitor<V> {
+ Stack<DepthString> depthString = new Stack<DepthString>();
+
+ @Override
+ public void visit(Stack<V> stack, V vertex) {
+ depthString.push(new DepthString(stack.size(), vertex.toString()));
+ }
+
+ public Stack<DepthString> getDepthStrings() {
+ return depthString;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java
index df4b046..e2724e7 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/BinaryNode.java
@@ -26,7 +26,7 @@ import org.apache.tajo.json.GsonObject;
public abstract class BinaryNode extends LogicalNode implements Cloneable, GsonObject {
@Expose LogicalNode leftChild = null;
- @Expose LogicalNode inner = null;
+ @Expose LogicalNode rightChild = null;
public BinaryNode() {
super();
@@ -39,27 +39,27 @@ public abstract class BinaryNode extends LogicalNode implements Cloneable, GsonO
super(opType);
}
- public LogicalNode getLeftChild() {
- return this.leftChild;
+ public <T extends LogicalNode> T getLeftChild() {
+ return (T) this.leftChild;
}
public void setLeftChild(LogicalNode op) {
this.leftChild = op;
}
- public LogicalNode getRightChild() {
- return this.inner;
+ public <T extends LogicalNode> T getRightChild() {
+ return (T) this.rightChild;
}
public void setRightChild(LogicalNode op) {
- this.inner = op;
+ this.rightChild = op;
}
@Override
public Object clone() throws CloneNotSupportedException {
BinaryNode binNode = (BinaryNode) super.clone();
binNode.leftChild = (LogicalNode) leftChild.clone();
- binNode.inner = (LogicalNode) inner.clone();
+ binNode.rightChild = (LogicalNode) rightChild.clone();
return binNode;
}
@@ -67,12 +67,12 @@ public abstract class BinaryNode extends LogicalNode implements Cloneable, GsonO
public void preOrder(LogicalNodeVisitor visitor) {
visitor.visit(this);
leftChild.postOrder(visitor);
- inner.postOrder(visitor);
+ rightChild.postOrder(visitor);
}
public void postOrder(LogicalNodeVisitor visitor) {
leftChild.postOrder(visitor);
- inner.postOrder(visitor);
+ rightChild.postOrder(visitor);
visitor.visit(this);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
index c06623b..f7ff8ef 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/CreateTableNode.java
@@ -24,6 +24,7 @@ import org.apache.tajo.catalog.Column;
import org.apache.tajo.catalog.Options;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.catalog.proto.CatalogProtos.StoreType;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.util.TUtil;
public class CreateTableNode extends LogicalNode implements Cloneable {
@@ -88,6 +89,12 @@ public class CreateTableNode extends LogicalNode implements Cloneable {
public void setExternal(boolean external) {
this.external = external;
}
+
+
+ @Override
+ public PlanString getPlanString() {
+ return new PlanString("CreateTable");
+ }
@Override
public boolean equals(Object obj) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java
index 7ddb5ef..708c03a 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/DropTableNode.java
@@ -18,6 +18,8 @@
package org.apache.tajo.engine.planner.logical;
+import org.apache.tajo.engine.planner.PlanString;
+
public class DropTableNode extends LogicalNode {
private String tableName;
@@ -30,6 +32,11 @@ public class DropTableNode extends LogicalNode {
return this.tableName;
}
+ @Override
+ public PlanString getPlanString() {
+ return new PlanString("DropTable");
+ }
+
public boolean equals(Object obj) {
if (obj instanceof DropTableNode) {
DropTableNode other = (DropTableNode) obj;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java
index 286f1c9..bb323dc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/EvalExprNode.java
@@ -22,6 +22,7 @@
package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.engine.planner.Target;
import org.apache.tajo.util.TUtil;
@@ -89,4 +90,9 @@ public class EvalExprNode extends LogicalNode implements Projectable {
public void postOrder(LogicalNodeVisitor visitor) {
// nothing
}
+
+ @Override
+ public PlanString getPlanString() {
+ return null;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java
index e7c7271..fd3c0d0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ExceptNode.java
@@ -21,6 +21,8 @@
*/
package org.apache.tajo.engine.planner.logical;
+import org.apache.tajo.engine.planner.PlanString;
+
public class ExceptNode extends BinaryNode {
public ExceptNode() {
@@ -33,6 +35,15 @@ public class ExceptNode extends BinaryNode {
setRightChild(inner);
}
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Except");
+ planStr.appendTitle(" (L - " + ((TableSubQueryNode)getLeftChild()).getTableName());
+ planStr.appendTitle(", R - " + ((TableSubQueryNode)getRightChild()).getTableName());
+ planStr.appendTitle(")");
+ return planStr;
+ }
+
public String toString() {
return getLeftChild().toString() + "\n EXCEPT \n" + getRightChild().toString();
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java
index c6b9bc8..34da374 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/GroupbyNode.java
@@ -21,6 +21,7 @@ package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.engine.planner.Target;
import org.apache.tajo.util.TUtil;
@@ -43,6 +44,10 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable {
this(columns);
this.havingCondition = havingCondition;
}
+
+ public final boolean isEmptyGrouping() {
+ return columns == null || columns.length == 0;
+ }
public final Column [] getGroupingColumns() {
return this.columns;
@@ -143,4 +148,33 @@ public class GroupbyNode extends UnaryNode implements Projectable, Cloneable {
return grp;
}
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Aggregation");
+
+ StringBuilder sb = new StringBuilder("Targets: ");
+ for (int i = 0; i < targets.length; i++) {
+ sb.append(targets[i]);
+ if( i < targets.length - 1) {
+ sb.append(",");
+ }
+ }
+ planStr.addExplan(sb.toString());
+
+ sb = new StringBuilder("Groups: ");
+ sb.append("(");
+ Column [] groupingColumns = columns;
+ for (int j = 0; j < groupingColumns.length; j++) {
+ sb.append(groupingColumns[j].getColumnName());
+ if(j < groupingColumns.length - 1) {
+ sb.append(",");
+ }
+ }
+
+ sb.append(")");
+
+ planStr.addExplan(sb.toString());
+ return planStr;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java
index 8a9ffe4..0cfbfb5 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IndexScanNode.java
@@ -29,7 +29,6 @@ public class IndexScanNode extends ScanNode {
@Expose private SortSpec [] sortKeys;
@Expose private Schema keySchema = null;
@Expose private Datum[] datum = null;
- //TODO- @Expose private IndexType type;
public IndexScanNode(ScanNode scanNode ,
Schema keySchema , Datum[] datum, SortSpec[] sortKeys ) {
@@ -38,7 +37,6 @@ public class IndexScanNode extends ScanNode {
setFromTable(scanNode.getFromTable());
setInSchema(scanNode.getInSchema());
setOutSchema(scanNode.getOutSchema());
- setLocal(scanNode.isLocal());
setTargets(scanNode.getTargets());
setType(NodeType.BST_INDEX_SCAN);
this.sortKeys = sortKeys;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java
index 870d3bd..0882e56 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/IntersectNode.java
@@ -21,6 +21,8 @@
*/
package org.apache.tajo.engine.planner.logical;
+import org.apache.tajo.engine.planner.PlanString;
+
public class IntersectNode extends BinaryNode {
public IntersectNode() {
@@ -33,6 +35,15 @@ public class IntersectNode extends BinaryNode {
setRightChild(inner);
}
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Intersect");
+ planStr.appendTitle(" (L - " + ((TableSubQueryNode)getLeftChild()).getTableName());
+ planStr.appendTitle(", R - " + ((TableSubQueryNode)getRightChild()).getTableName());
+ planStr.appendTitle(")");
+ return planStr;
+ }
+
public String toString() {
return getLeftChild().toString() + "\n INTERSECT \n" + getRightChild().toString();
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java
index 6132c90..173ad55 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/JoinNode.java
@@ -24,6 +24,7 @@ package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
import org.apache.tajo.algebra.JoinType;
import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.engine.planner.Target;
public class JoinNode extends BinaryNode implements Projectable, Cloneable {
@@ -80,11 +81,21 @@ public class JoinNode extends BinaryNode implements Projectable, Cloneable {
}
@Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Join (type : ")
+ .appendTitle(joinType +")");
+ if (hasJoinQual()) {
+ planStr.addExplan("Join Cond: " + joinQual.toString());
+ }
+ return planStr;
+ }
+
+ @Override
public boolean equals(Object obj) {
if (obj instanceof JoinNode) {
JoinNode other = (JoinNode) obj;
return super.equals(other) && leftChild.equals(other.leftChild)
- && inner.equals(other.inner);
+ && rightChild.equals(other.rightChild);
} else {
return false;
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java
index d5bc926..697f1fc 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LimitNode.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.planner.PlanString;
public final class LimitNode extends UnaryNode implements Cloneable {
@Expose private long fetchFirstNum;
@@ -35,6 +36,11 @@ public final class LimitNode extends UnaryNode implements Cloneable {
public long getFetchFirstNum() {
return fetchFirstNum;
}
+
+ @Override
+ public PlanString getPlanString() {
+ return new PlanString("Limit");
+ }
@Override
public boolean equals(Object obj) {
@@ -56,10 +62,10 @@ public final class LimitNode extends UnaryNode implements Cloneable {
}
public String toString() {
- StringBuilder sb = new StringBuilder("LIMIT ").append(fetchFirstNum);
+ StringBuilder sb = new StringBuilder("Limit (").append(fetchFirstNum).append(")");
- sb.append("\n\"out schema: ").append(getOutSchema())
- .append("\n\"in schema: " + getInSchema());
+ sb.append("\n \"out schema: ").append(getOutSchema())
+ .append("\n \"in schema: " + getInSchema());
sb.append("\n").append(getChild().toString());
return sb.toString();
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java
index 6daaa50..7d7e86d 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalNode.java
@@ -22,9 +22,10 @@
package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
-import org.apache.tajo.json.GsonObject;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.engine.json.CoreGsonHelper;
+import org.apache.tajo.engine.planner.PlanString;
+import org.apache.tajo.json.GsonObject;
import org.apache.tajo.util.TUtil;
public abstract class LogicalNode implements Cloneable, GsonObject {
@@ -35,7 +36,6 @@ public abstract class LogicalNode implements Cloneable, GsonObject {
@Expose private double cost = 0;
public LogicalNode() {
-
}
public LogicalNode(NodeType type) {
@@ -109,4 +109,6 @@ public abstract class LogicalNode implements Cloneable, GsonObject {
public abstract void preOrder(LogicalNodeVisitor visitor);
public abstract void postOrder(LogicalNodeVisitor visitor);
+
+ public abstract PlanString getPlanString();
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java
index 67c08a6..e5373db 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/LogicalRootNode.java
@@ -18,6 +18,8 @@
package org.apache.tajo.engine.planner.logical;
+import org.apache.tajo.engine.planner.PlanString;
+
public class LogicalRootNode extends UnaryNode implements Cloneable {
public LogicalRootNode() {
super(NodeType.ROOT);
@@ -44,4 +46,9 @@ public class LogicalRootNode extends UnaryNode implements Cloneable {
public Object clone() throws CloneNotSupportedException {
return super.clone();
}
+
+ @Override
+ public PlanString getPlanString() {
+ return new PlanString("Root");
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java
index 35171ef..792fee8 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java
@@ -45,7 +45,9 @@ public enum NodeType {
SELECTION(SelectionNode.class),
STORE(StoreTableNode.class),
SORT(SortNode.class),
- UNION(UnionNode.class);
+ UNION(UnionNode.class),
+ TABLE_SUBQUERY(TableSubQueryNode.class);
+
private final Class<? extends LogicalNode> baseClass;
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java
index 90891bc..935f188 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ProjectionNode.java
@@ -19,6 +19,7 @@
package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.engine.planner.Target;
import java.util.Arrays;
@@ -108,4 +109,31 @@ public class ProjectionNode extends UnaryNode implements Projectable {
return projNode;
}
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Projection: ");
+
+ if (distinct) {
+ planStr.appendTitle(" (distinct)");
+ }
+
+
+ StringBuilder sb = new StringBuilder("Targets: ");
+ for (int i = 0; i < targets.length; i++) {
+ sb.append(targets[i]);
+ if( i < targets.length - 1) {
+ sb.append(", ");
+ }
+ }
+ planStr.addExplan(sb.toString());
+ if (getOutSchema() != null) {
+ planStr.addExplan("out schema: " + getOutSchema().toString());
+ }
+ if (getInSchema() != null) {
+ planStr.addExplan("in schema: " + getInSchema().toString());
+ }
+
+ return planStr;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ReceiveNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ReceiveNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ReceiveNode.java
index 1e566b0..a37ad5f 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ReceiveNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ReceiveNode.java
@@ -27,9 +27,7 @@ import com.google.common.collect.Maps;
import com.google.gson.Gson;
import com.google.gson.annotations.Expose;
import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.engine.planner.logical.NodeType;
-import org.apache.tajo.engine.planner.logical.LogicalNode;
-import org.apache.tajo.engine.planner.logical.LogicalNodeVisitor;
+import org.apache.tajo.engine.planner.PlanString;
import java.net.URI;
import java.util.*;
@@ -74,6 +72,12 @@ public final class ReceiveNode extends LogicalNode implements Cloneable {
return Collections.unmodifiableSet(fetchMap.entrySet());
}
+
+ @Override
+ public PlanString getPlanString() {
+ return new PlanString("Receive ");
+ }
+
@Override
public boolean equals(Object obj) {
if (obj instanceof ReceiveNode) {
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java
new file mode 100644
index 0000000..583e472
--- /dev/null
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/RelationNode.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tajo.engine.planner.logical;
+
+import org.apache.tajo.catalog.Schema;
+
+public abstract class RelationNode extends LogicalNode {
+
+ public RelationNode(NodeType nodeType) {
+ super(nodeType);
+ assert(nodeType == NodeType.SCAN || nodeType == NodeType.TABLE_SUBQUERY);
+ }
+
+ public abstract String getTableName();
+
+ public abstract String getCanonicalName();
+
+ public abstract Schema getTableSchema();
+}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
index 584a697..6e528b0 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/ScanNode.java
@@ -23,19 +23,17 @@ import com.google.gson.annotations.Expose;
import org.apache.tajo.catalog.Schema;
import org.apache.tajo.engine.eval.EvalNode;
import org.apache.tajo.engine.planner.FromTable;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.engine.planner.Target;
import org.apache.tajo.util.TUtil;
-public class ScanNode extends LogicalNode implements Projectable {
+public class ScanNode extends RelationNode implements Projectable {
@Expose private FromTable table;
@Expose private EvalNode qual;
@Expose private Target[] targets;
- @Expose private boolean local;
- @Expose private boolean broadcast;
public ScanNode() {
- super();
- local = false;
+ super(NodeType.SCAN);
}
public ScanNode(FromTable table) {
@@ -43,10 +41,9 @@ public class ScanNode extends LogicalNode implements Projectable {
this.table = table;
this.setInSchema(table.getSchema());
this.setOutSchema(table.getSchema());
- local = false;
}
- public String getTableId() {
+ public String getTableName() {
return table.getTableName();
}
@@ -70,14 +67,6 @@ public class ScanNode extends LogicalNode implements Projectable {
return this.qual;
}
- public boolean isLocal() {
- return this.local;
- }
-
- public void setLocal(boolean local) {
- this.local = local;
- }
-
public void setQual(EvalNode evalTree) {
this.qual = evalTree;
}
@@ -96,14 +85,6 @@ public class ScanNode extends LogicalNode implements Projectable {
public Target [] getTargets() {
return this.targets;
}
-
- public boolean isBroadcast() {
- return broadcast;
- }
-
- public void setBroadcast() {
- broadcast = true;
- }
public FromTable getFromTable() {
return this.table;
@@ -120,10 +101,6 @@ public class ScanNode extends LogicalNode implements Projectable {
if (hasAlias()) {
sb.append(",\"alias\": \"").append(table.getAlias());
}
-
- if (isBroadcast()) {
- sb.append(",\"broadcast\": true\"");
- }
if (hasQual()) {
sb.append(", \"qual\": \"").append(this.qual).append("\"");
@@ -196,4 +173,33 @@ public class ScanNode extends LogicalNode implements Projectable {
public void postOrder(LogicalNodeVisitor visitor) {
visitor.visit(this);
}
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Scan on ").appendTitle(getTableName());
+ if (table.hasAlias()) {
+ planStr.appendTitle(" as ").appendTitle(table.getAlias());
+ }
+
+ if (hasQual()) {
+ planStr.addExplan("filter: ").appendExplain(this.qual.toString());
+ }
+
+ if (hasTargets()) {
+ planStr.addExplan("target list: ");
+ boolean first = true;
+ for (Target target : targets) {
+ if (!first) {
+ planStr.appendExplain(", ");
+ }
+ planStr.appendExplain(target.toString());
+ first = false;
+ }
+ }
+
+ planStr.addDetail("out schema: ").appendDetail(getOutSchema().toString());
+ planStr.addDetail("in schema: ").appendDetail(getInSchema().toString());
+
+ return planStr;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java
index 2b2a0ac..2435453 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SelectionNode.java
@@ -20,6 +20,7 @@ package org.apache.tajo.engine.planner.logical;
import com.google.gson.annotations.Expose;
import org.apache.tajo.engine.eval.EvalNode;
+import org.apache.tajo.engine.planner.PlanString;
public class SelectionNode extends UnaryNode implements Cloneable {
@Expose private EvalNode qual;
@@ -40,17 +41,14 @@ public class SelectionNode extends UnaryNode implements Cloneable {
public void setQual(EvalNode qual) {
this.qual = qual;
}
-
- public String toString() {
- StringBuilder sb = new StringBuilder();
- sb.append("\"Selection\": {\"qual\": \"").append(qual.toString()).append("\",");
- sb.append("\n \"out schema\": ").append(getOutSchema()).append(",");
- sb.append("\n \"in schema\": ").append(getInSchema()).append("}");
-
- return sb.toString()+"\n"
- + getChild().toString();
+
+ @Override
+ public PlanString getPlanString() {
+ PlanString planStr = new PlanString("Filter");
+ planStr.addExplan("Search Cond: " + getQual());
+ return planStr;
}
-
+
@Override
public boolean equals(Object obj) {
if (obj instanceof SelectionNode) {
@@ -70,4 +68,14 @@ public class SelectionNode extends UnaryNode implements Cloneable {
return selNode;
}
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("\"Selection\": {\"qual\": \"").append(qual.toString()).append("\",");
+ sb.append("\n \"out schema\": ").append(getOutSchema()).append(",");
+ sb.append("\n \"in schema\": ").append(getInSchema()).append("}");
+
+ return sb.toString()+"\n"
+ + getChild().toString();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-tajo/blob/1b1d1e8c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SendNode.java
----------------------------------------------------------------------
diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SendNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SendNode.java
index 5456a77..96e6fc5 100644
--- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SendNode.java
+++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/SendNode.java
@@ -28,8 +28,7 @@ import com.google.gson.Gson;
import com.google.gson.annotations.Expose;
import org.apache.tajo.catalog.Column;
import org.apache.tajo.engine.json.CoreGsonHelper;
-import org.apache.tajo.engine.planner.logical.NodeType;
-import org.apache.tajo.engine.planner.logical.UnaryNode;
+import org.apache.tajo.engine.planner.PlanString;
import org.apache.tajo.util.TUtil;
import java.net.URI;
@@ -108,6 +107,11 @@ public class SendNode extends UnaryNode {
}
@Override
+ public PlanString getPlanString() {
+ return null;
+ }
+
+ @Override
public boolean equals(Object obj) {
if (obj instanceof SendNode) {
SendNode other = (SendNode) obj;