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 2014/04/21 13:12:55 UTC
[1/9] git commit: TAJO-763: Out of range problem in utc_usec_to().
(hyunsik)
Repository: tajo
Updated Branches:
refs/heads/window_function 1b3ec373c -> 39d4f2d7b
TAJO-763: Out of range problem in utc_usec_to(). (hyunsik)
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/92e9a667
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/92e9a667
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/92e9a667
Branch: refs/heads/window_function
Commit: 92e9a667379b99df1e9160ac207246ac65b0cfac
Parents: f8ba4db
Author: Hyunsik Choi <hy...@apache.org>
Authored: Sat Apr 19 13:40:02 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Sat Apr 19 13:40:02 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 2 +
.../org/apache/tajo/util/TimeStampUtil.java | 11 +--
.../org/apache/tajo/util/TestTimeStampUtil.java | 75 ++++++++++++++++++++
3 files changed, 84 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/92e9a667/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1bc79c9..46f78c0 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -334,6 +334,8 @@ Release 0.8.0 - unreleased
BUG FIXES
+ TAJO-763: Out of range problem in utc_usec_to(). (hyunsik)
+
TAJO-741: GreedyHeuristicJoinOrderAlgorithm removes some join pairs. (jaehwa)
TAJO-772: TajoDump cannot dump upper/lower mixed case database names.
http://git-wip-us.apache.org/repos/asf/tajo/blob/92e9a667/tajo-common/src/main/java/org/apache/tajo/util/TimeStampUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/TimeStampUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/TimeStampUtil.java
index 31c5b90..830c4aa 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/TimeStampUtil.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/TimeStampUtil.java
@@ -30,19 +30,22 @@ public class TimeStampUtil {
}
public static long getHour(DateTime dateTime) {
- return convertToMicroSeconds(dateTime.withTime(dateTime.get(DateTimeFieldType.clockhourOfDay()), 0, 0, 0));
+ return convertToMicroSeconds(dateTime.withTime(dateTime.get(DateTimeFieldType.hourOfDay()), 0, 0, 0));
}
public static long getMinute(DateTime dateTime) {
- return convertToMicroSeconds(dateTime.withTime(dateTime.get(DateTimeFieldType.clockhourOfDay()), dateTime.get(DateTimeFieldType.minuteOfHour()), 0, 0));
+ return convertToMicroSeconds(dateTime.withTime(dateTime.get(DateTimeFieldType.hourOfDay()),
+ dateTime.get(DateTimeFieldType.minuteOfHour()), 0, 0));
}
public static long getSecond(DateTime dateTime) {
- return convertToMicroSeconds(dateTime.withTime(dateTime.get(DateTimeFieldType.clockhourOfDay()), dateTime.get(DateTimeFieldType.minuteOfHour()), dateTime.get(DateTimeFieldType.secondOfMinute()), 0));
+ return convertToMicroSeconds(dateTime.withTime(dateTime.get(DateTimeFieldType.hourOfDay()),
+ dateTime.get(DateTimeFieldType.minuteOfHour()), dateTime.get(DateTimeFieldType.secondOfMinute()), 0));
}
public static long getMonth(DateTime dateTime) {
- return convertToMicroSeconds(dateTime.withTimeAtStartOfDay().withDate(dateTime.getYear(),dateTime.getMonthOfYear(),1));
+ return convertToMicroSeconds(dateTime.withTimeAtStartOfDay().withDate(dateTime.getYear(),
+ dateTime.getMonthOfYear(),1));
}
public static long getDayOfWeek(DateTime dateTime,int week) {
http://git-wip-us.apache.org/repos/asf/tajo/blob/92e9a667/tajo-common/src/test/java/org/apache/tajo/util/TestTimeStampUtil.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/test/java/org/apache/tajo/util/TestTimeStampUtil.java b/tajo-common/src/test/java/org/apache/tajo/util/TestTimeStampUtil.java
new file mode 100644
index 0000000..353063d
--- /dev/null
+++ b/tajo-common/src/test/java/org/apache/tajo/util/TestTimeStampUtil.java
@@ -0,0 +1,75 @@
+/**
+ * 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.util;
+
+import org.joda.time.DateTimeZone;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTimeStampUtil {
+ private static final int TEST_YEAR = 2014;
+ private static final int TEST_MONTH_OF_YEAR = 4;
+ private static final int TEST_DAY_OF_MONTH = 18;
+ private static final int TEST_HOUR_OF_DAY = 0;
+ private static final int TEST_MINUTE_OF_HOUR = 15;
+ private static final int TEST_SECOND_OF_MINUTE = 25;
+ private static final DateTime TEST_DATETIME = new DateTime(TEST_YEAR, TEST_MONTH_OF_YEAR, TEST_DAY_OF_MONTH,
+ TEST_HOUR_OF_DAY, TEST_MINUTE_OF_HOUR, TEST_SECOND_OF_MINUTE, DateTimeZone.UTC);
+ private static final DateTimeFormatter fmt = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss");
+
+ @Test
+ public void testGetYear() {
+ assertEquals(DateTime.parse("2014-01-01 00:00:00", fmt.withZoneUTC()).getMillis() * 1000,
+ TimeStampUtil.getYear(TEST_DATETIME));
+ }
+
+ @Test
+ public void testGetMonth() {
+ assertEquals(DateTime.parse("2014-04-01 00:00:00", fmt.withZoneUTC()).getMillis() * 1000,
+ TimeStampUtil.getMonth(TEST_DATETIME));
+ }
+
+ @Test
+ public void testGetDay() {
+ assertEquals(DateTime.parse("2014-04-18 00:00:00", fmt.withZoneUTC()).getMillis() * 1000,
+ TimeStampUtil.getDay(TEST_DATETIME));
+ }
+
+ @Test
+ public void testGetHour() {
+ assertEquals(DateTime.parse("2014-04-18 00:00:00",fmt.withZoneUTC()).getMillis() * 1000,
+ TimeStampUtil.getHour(TEST_DATETIME));
+ }
+
+ @Test
+ public void testGetMinute() {
+ assertEquals(DateTime.parse("2014-04-18 00:15:00",fmt.withZoneUTC()).getMillis() * 1000,
+ TimeStampUtil.getMinute(TEST_DATETIME));
+ }
+
+ @Test
+ public void testGetSecond() {
+ assertEquals(DateTime.parse("2014-04-18 00:15:25",fmt.withZoneUTC()).getMillis() * 1000,
+ TimeStampUtil.getSecond(TEST_DATETIME));
+ }
+}
[2/9] git commit: TAJO-763: Out of range problem in utc_usec_to()
(modified contributor name to Ilhyun Suh in CHANGES.txt).
Posted by hy...@apache.org.
TAJO-763: Out of range problem in utc_usec_to() (modified contributor name to Ilhyun Suh in CHANGES.txt).
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/6a005ce7
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/6a005ce7
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/6a005ce7
Branch: refs/heads/window_function
Commit: 6a005ce7e4c5ed80278da702255197936c189ff3
Parents: 92e9a66
Author: Hyunsik Choi <hy...@apache.org>
Authored: Sat Apr 19 13:42:30 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Sat Apr 19 13:42:30 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/6a005ce7/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 46f78c0..94b6a84 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -334,7 +334,7 @@ Release 0.8.0 - unreleased
BUG FIXES
- TAJO-763: Out of range problem in utc_usec_to(). (hyunsik)
+ TAJO-763: Out of range problem in utc_usec_to(). (Ilhyun Suh via hyunsik)
TAJO-741: GreedyHeuristicJoinOrderAlgorithm removes some join pairs. (jaehwa)
[8/9] git commit: TAJO-652: logical planner cannot handle alias on
partition columns. (Hyoungjun Kim via hyunsik)
Posted by hy...@apache.org.
TAJO-652: logical planner cannot handle alias on partition columns. (Hyoungjun Kim via hyunsik)
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/cbe1d6e9
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/cbe1d6e9
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/cbe1d6e9
Branch: refs/heads/window_function
Commit: cbe1d6e94ed7e3bdcd30d0414815736621b6b8df
Parents: bbf2461
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Apr 21 18:24:41 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Apr 21 18:26:11 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 3 +
.../apache/tajo/engine/planner/LogicalPlan.java | 230 ++++++++++---------
.../tajo/engine/query/TestTablePartitions.java | 15 ++
.../queries/TestTablePartitions/case11.sql | 1 +
.../queries/TestTablePartitions/case12.sql | 7 +
.../queries/TestTablePartitions/case13.sql | 11 +
.../results/TestTablePartitions/case11.result | 7 +
.../results/TestTablePartitions/case12.result | 7 +
.../results/TestTablePartitions/case13.result | 7 +
.../org/apache/tajo/rpc/NettyServerBase.java | 3 +
10 files changed, 184 insertions(+), 107 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7f1b175..eaf81a5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -336,6 +336,9 @@ Release 0.8.0 - unreleased
BUG FIXES
+ TAJO-652: logical planner cannot handle alias on partition columns.
+ (Hyoungjun Kim via hyunsik)
+
TAJO-675: maximum frame size of frameDecoder should be increased. (jinho)
TAJO-748: Shuffle output numbers of join may be inconsistent. (jaehwa)
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
index 98fbf42..6be0c6a 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/LogicalPlan.java
@@ -286,105 +286,86 @@ public class LogicalPlan {
* It resolves a column.
*/
public Column resolveColumn(QueryBlock block, ColumnReferenceExpr columnRef) throws PlanningException {
+ if (columnRef.hasQualifier()) {
+ return resolveColumnWithQualifier(block, columnRef);
+ } else {
+ return resolveColumnWithoutQualifier(block, columnRef);
+ }
+ }
- if (columnRef.hasQualifier()) { // if a column reference is qualified
-
- String qualifier;
- String canonicalName;
- String qualifiedName;
+ private Column resolveColumnWithQualifier(QueryBlock block, ColumnReferenceExpr columnRef) throws PlanningException {
+ String qualifier;
+ String canonicalName;
+ String qualifiedName;
- if (CatalogUtil.isFQTableName(columnRef.getQualifier())) {
- qualifier = columnRef.getQualifier();
- canonicalName = columnRef.getCanonicalName();
- } else {
- String resolvedDatabaseName = resolveDatabase(block, columnRef.getQualifier());
- if (resolvedDatabaseName == null) {
- throw new NoSuchColumnException(columnRef.getQualifier());
- }
- qualifier = CatalogUtil.buildFQName(resolvedDatabaseName, columnRef.getQualifier());
- canonicalName = CatalogUtil.buildFQName(qualifier, columnRef.getName());
+ if (CatalogUtil.isFQTableName(columnRef.getQualifier())) {
+ qualifier = columnRef.getQualifier();
+ canonicalName = columnRef.getCanonicalName();
+ } else {
+ String resolvedDatabaseName = resolveDatabase(block, columnRef.getQualifier());
+ if (resolvedDatabaseName == null) {
+ throw new NoSuchColumnException(columnRef.getQualifier());
}
- qualifiedName = CatalogUtil.buildFQName(qualifier, columnRef.getName());
+ qualifier = CatalogUtil.buildFQName(resolvedDatabaseName, columnRef.getQualifier());
+ canonicalName = CatalogUtil.buildFQName(qualifier, columnRef.getName());
+ }
+ qualifiedName = CatalogUtil.buildFQName(qualifier, columnRef.getName());
- RelationNode relationOp = block.getRelation(qualifier);
+ RelationNode relationOp = block.getRelation(qualifier);
- // if a column name is outside of this query block
- if (relationOp == null) {
- // TODO - nested query can only refer outer query block? or not?
- for (QueryBlock eachBlock : queryBlocks.values()) {
- if (eachBlock.existsRelation(qualifier)) {
- relationOp = eachBlock.getRelation(qualifier);
- }
+ // if a column name is outside of this query block
+ if (relationOp == null) {
+ // TODO - nested query can only refer outer query block? or not?
+ for (QueryBlock eachBlock : queryBlocks.values()) {
+ if (eachBlock.existsRelation(qualifier)) {
+ relationOp = eachBlock.getRelation(qualifier);
}
}
+ }
- // If we cannot find any relation against a qualified column name
- if (relationOp == null) {
- throw new NoSuchColumnException(canonicalName);
- }
-
- if (block.isAlreadyRenamedTableName(CatalogUtil.extractQualifier(canonicalName))) {
- String changedName = CatalogUtil.buildFQName(
- relationOp.getCanonicalName(),
- CatalogUtil.extractSimpleName(canonicalName));
- canonicalName = changedName;
- }
-
- Schema schema = relationOp.getTableSchema();
- Column column = schema.getColumn(canonicalName);
- if (column == null) {
- throw new NoSuchColumnException(canonicalName);
- }
+ // If we cannot find any relation against a qualified column name
+ if (relationOp == null) {
+ throw new NoSuchColumnException(canonicalName);
+ }
- // If code reach here, a column is found.
- // But, it may be aliased from bottom logical node.
- // If the column is aliased, the found name may not be used in upper node.
-
- // Here, we try to check if column reference is already aliased.
- // If so, it replaces the name with aliased name.
- LogicalNode currentNode = block.getCurrentNode();
-
- // The condition (currentNode.getInSchema().contains(column)) means
- // the column can be used at the current node. So, we don't need to find aliase name.
- if (currentNode != null && !currentNode.getInSchema().contains(column)
- && currentNode.getType() != NodeType.TABLE_SUBQUERY) {
- List<Column> candidates = TUtil.newList();
- if (block.namedExprsMgr.isAliased(qualifiedName)) {
- String alias = block.namedExprsMgr.getAlias(canonicalName);
- Column found = resolveColumn(block, new ColumnReferenceExpr(alias));
- if (found != null) {
- candidates.add(found);
- }
- }
- if (!candidates.isEmpty()) {
- return ensureUniqueColumn(candidates);
- }
- }
+ if (block.isAlreadyRenamedTableName(CatalogUtil.extractQualifier(canonicalName))) {
+ String changedName = CatalogUtil.buildFQName(
+ relationOp.getCanonicalName(),
+ CatalogUtil.extractSimpleName(canonicalName));
+ canonicalName = changedName;
+ }
- return column;
- } else { // if a column reference is not qualified
+ Schema schema = relationOp.getTableSchema();
+ Column column = schema.getColumn(canonicalName);
+ if (column == null) {
+ throw new NoSuchColumnException(canonicalName);
+ }
- // Trying to find the column within the current block
+ // If code reach here, a column is found.
+ // But, it may be aliased from bottom logical node.
+ // If the column is aliased, the found name may not be used in upper node.
- if (block.currentNode != null && block.currentNode.getInSchema() != null) {
- Column found = block.currentNode.getInSchema().getColumn(columnRef.getCanonicalName());
- if (found != null) {
- return found;
- }
- }
+ // Here, we try to check if column reference is already aliased.
+ // If so, it replaces the name with aliased name.
+ LogicalNode currentNode = block.getCurrentNode();
- if (block.getLatestNode() != null) {
- Column found = block.getLatestNode().getOutSchema().getColumn(columnRef.getName());
- if (found != null) {
- return found;
- }
+ // The condition (currentNode.getInSchema().contains(column)) means
+ // the column can be used at the current node. So, we don't need to find aliase name.
+ Schema currentNodeSchema = null;
+ if (currentNode != null) {
+ if (currentNode instanceof RelationNode) {
+ currentNodeSchema = ((RelationNode) currentNode).getTableSchema();
+ } else {
+ currentNodeSchema = currentNode.getInSchema();
}
+ }
+ if (currentNode != null && !currentNodeSchema.contains(column)
+ && currentNode.getType() != NodeType.TABLE_SUBQUERY) {
List<Column> candidates = TUtil.newList();
- // Trying to find columns from aliased references.
- if (block.namedExprsMgr.isAliased(columnRef.getCanonicalName())) {
- String originalName = block.namedExprsMgr.getAlias(columnRef.getCanonicalName());
- Column found = resolveColumn(block, new ColumnReferenceExpr(originalName));
+ if (block.namedExprsMgr.isAliased(qualifiedName)) {
+ String alias = block.namedExprsMgr.getAlias(canonicalName);
+ Column found = resolveColumn(block, new ColumnReferenceExpr(alias));
if (found != null) {
candidates.add(found);
}
@@ -392,47 +373,80 @@ public class LogicalPlan {
if (!candidates.isEmpty()) {
return ensureUniqueColumn(candidates);
}
+ }
- // Trying to find columns from other relations in the current block
- for (RelationNode rel : block.getRelations()) {
- Column found = rel.getTableSchema().getColumn(columnRef.getName());
- if (found != null) {
- candidates.add(found);
- }
+ return column;
+ }
+
+ private Column resolveColumnWithoutQualifier(QueryBlock block,
+ ColumnReferenceExpr columnRef)throws PlanningException {
+ // Trying to find the column within the current block
+ if (block.currentNode != null && block.currentNode.getInSchema() != null) {
+ Column found = block.currentNode.getInSchema().getColumn(columnRef.getCanonicalName());
+ if (found != null) {
+ return found;
}
+ }
- if (!candidates.isEmpty()) {
- return ensureUniqueColumn(candidates);
+ if (block.getLatestNode() != null) {
+ Column found = block.getLatestNode().getOutSchema().getColumn(columnRef.getName());
+ if (found != null) {
+ return found;
}
+ }
- // Trying to find columns from other relations in other blocks
- for (QueryBlock eachBlock : queryBlocks.values()) {
- for (RelationNode rel : eachBlock.getRelations()) {
- Column found = rel.getTableSchema().getColumn(columnRef.getName());
- if (found != null) {
- candidates.add(found);
- }
- }
+ List<Column> candidates = TUtil.newList();
+ // Trying to find columns from aliased references.
+ if (block.namedExprsMgr.isAliased(columnRef.getCanonicalName())) {
+ String originalName = block.namedExprsMgr.getAlias(columnRef.getCanonicalName());
+ Column found = resolveColumn(block, new ColumnReferenceExpr(originalName));
+ if (found != null) {
+ candidates.add(found);
}
+ }
+ if (!candidates.isEmpty()) {
+ return ensureUniqueColumn(candidates);
+ }
- if (!candidates.isEmpty()) {
- return ensureUniqueColumn(candidates);
+ // Trying to find columns from other relations in the current block
+ for (RelationNode rel : block.getRelations()) {
+ Column found = rel.getTableSchema().getColumn(columnRef.getName());
+ if (found != null) {
+ candidates.add(found);
}
+ }
- // Trying to find columns from schema in current block.
- if (block.getSchema() != null) {
- Column found = block.getSchema().getColumn(columnRef.getName());
+ if (!candidates.isEmpty()) {
+ return ensureUniqueColumn(candidates);
+ }
+
+ // Trying to find columns from other relations in other blocks
+ for (QueryBlock eachBlock : queryBlocks.values()) {
+ for (RelationNode rel : eachBlock.getRelations()) {
+ Column found = rel.getTableSchema().getColumn(columnRef.getName());
if (found != null) {
candidates.add(found);
}
}
+ }
- if (!candidates.isEmpty()) {
- return ensureUniqueColumn(candidates);
+ if (!candidates.isEmpty()) {
+ return ensureUniqueColumn(candidates);
+ }
+
+ // Trying to find columns from schema in current block.
+ if (block.getSchema() != null) {
+ Column found = block.getSchema().getColumn(columnRef.getName());
+ if (found != null) {
+ candidates.add(found);
}
+ }
- throw new VerifyException("ERROR: no such a column name "+ columnRef.getCanonicalName());
+ if (!candidates.isEmpty()) {
+ return ensureUniqueColumn(candidates);
}
+
+ throw new VerifyException("ERROR: no such a column name "+ columnRef.getCanonicalName());
}
private static Column ensureUniqueColumn(List<Column> candidates)
@@ -676,6 +690,7 @@ public class LogicalPlan {
}
}
+ @SuppressWarnings("unchecked")
public <T extends LogicalNode> T getCurrentNode() {
return (T) this.currentNode;
}
@@ -722,6 +737,7 @@ public class LogicalPlan {
queryBlockByPID.put(node.getPID(), this);
}
+ @SuppressWarnings("unchecked")
public <T extends LogicalNode> T getNode(NodeType nodeType) {
return (T) nodeTypeToNodeMap.get(nodeType);
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index 9cc2410..0ec7de0 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -187,6 +187,21 @@ public class TestTablePartitions extends QueryTestCaseBase {
res = executeFile("case10.sql");
assertResultSet(res, "case10.result");
res.close();
+
+ // alias partition column
+ res = executeFile("case11.sql");
+ assertResultSet(res, "case11.result");
+ res.close();
+
+ // alias partition column in group by, order by
+ res = executeFile("case12.sql");
+ assertResultSet(res, "case12.result");
+ res.close();
+
+ // alias partition column in subquery
+ res = executeFile("case13.sql");
+ assertResultSet(res, "case13.result");
+ res.close();
}
@Test
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/test/resources/queries/TestTablePartitions/case11.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case11.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case11.sql
new file mode 100644
index 0000000..a5bf1db
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case11.sql
@@ -0,0 +1 @@
+select key as key_alias from testQueryCasesOnColumnPartitionedTable
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/test/resources/queries/TestTablePartitions/case12.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case12.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case12.sql
new file mode 100644
index 0000000..2324439
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case12.sql
@@ -0,0 +1,7 @@
+select
+ key as key_alias,
+ count(*) cnt
+from
+ testQueryCasesOnColumnPartitionedTable
+group by key_alias
+order by key_alias desc
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/test/resources/queries/TestTablePartitions/case13.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case13.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case13.sql
new file mode 100644
index 0000000..efb7432
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case13.sql
@@ -0,0 +1,11 @@
+select
+ key_alias as key, cnt
+from (
+ select
+ key as key_alias,
+ count(*) cnt
+ from
+ testQueryCasesOnColumnPartitionedTable
+ group by key_alias
+ order by key_alias desc
+) a
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/test/resources/results/TestTablePartitions/case11.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case11.result b/tajo-core/src/test/resources/results/TestTablePartitions/case11.result
new file mode 100644
index 0000000..95750fd
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case11.result
@@ -0,0 +1,7 @@
+key_alias
+-------------------------------
+17.0
+36.0
+38.0
+45.0
+49.0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/test/resources/results/TestTablePartitions/case12.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case12.result b/tajo-core/src/test/resources/results/TestTablePartitions/case12.result
new file mode 100644
index 0000000..ac0f9b7
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case12.result
@@ -0,0 +1,7 @@
+key_alias,cnt
+-------------------------------
+49.0,1
+45.0,1
+38.0,1
+36.0,1
+17.0,1
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-core/src/test/resources/results/TestTablePartitions/case13.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case13.result b/tajo-core/src/test/resources/results/TestTablePartitions/case13.result
new file mode 100644
index 0000000..5c92e79
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case13.result
@@ -0,0 +1,7 @@
+key,cnt
+-------------------------------
+49.0,1
+45.0,1
+38.0,1
+36.0,1
+17.0,1
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/cbe1d6e9/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
index 8f98d3a..9ee098d 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/NettyServerBase.java
@@ -25,6 +25,7 @@ import org.jboss.netty.bootstrap.ServerBootstrap;
import org.jboss.netty.channel.Channel;
import org.jboss.netty.channel.ChannelFactory;
import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.DefaultChannelFuture;
import org.jboss.netty.channel.group.ChannelGroup;
import org.jboss.netty.channel.group.DefaultChannelGroup;
@@ -67,6 +68,8 @@ public class NettyServerBase {
public void init(ChannelPipelineFactory pipeline, int workerNum) {
ChannelFactory factory = RpcChannelFactory.createServerChannelFactory(serviceName, workerNum);
+ DefaultChannelFuture.setUseDeadLockChecker(false);
+
pipelineFactory = pipeline;
bootstrap = new ServerBootstrap(factory);
bootstrap.setPipelineFactory(pipelineFactory);
[4/9] git commit: TAJO-777: Partition column in function parameter
occurs NPE. (missed test files)
Posted by hy...@apache.org.
TAJO-777: Partition column in function parameter occurs NPE. (missed test files)
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/f1f36ec5
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/f1f36ec5
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/f1f36ec5
Branch: refs/heads/window_function
Commit: f1f36ec5f5ef78168dfe024c07c1ba122f2d78c6
Parents: e56a7a41
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Apr 21 11:53:25 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Apr 21 11:53:25 2014 +0900
----------------------------------------------------------------------
.../src/test/resources/queries/TestTablePartitions/case4.sql | 1 +
.../src/test/resources/queries/TestTablePartitions/case5.sql | 1 +
.../src/test/resources/queries/TestTablePartitions/case6.sql | 1 +
.../src/test/resources/queries/TestTablePartitions/case7.sql | 1 +
.../test/resources/results/TestTablePartitions/case4.result | 7 +++++++
.../test/resources/results/TestTablePartitions/case5.result | 7 +++++++
.../test/resources/results/TestTablePartitions/case6.result | 7 +++++++
7 files changed, 25 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/f1f36ec5/tajo-core/src/test/resources/queries/TestTablePartitions/case4.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case4.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case4.sql
new file mode 100644
index 0000000..477a456
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case4.sql
@@ -0,0 +1 @@
+select pow(key, 2) from testQueryCasesOnColumnPartitionedTable
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/f1f36ec5/tajo-core/src/test/resources/queries/TestTablePartitions/case5.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case5.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case5.sql
new file mode 100644
index 0000000..180dd72
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case5.sql
@@ -0,0 +1 @@
+select round(pow(key + 1, 2)) from testQueryCasesOnColumnPartitionedTable
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/f1f36ec5/tajo-core/src/test/resources/queries/TestTablePartitions/case6.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case6.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case6.sql
new file mode 100644
index 0000000..cd54486
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case6.sql
@@ -0,0 +1 @@
+select col1, key from testQueryCasesOnColumnPartitionedTable order by pow(key, 2) desc
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/f1f36ec5/tajo-core/src/test/resources/queries/TestTablePartitions/case7.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case7.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case7.sql
new file mode 100644
index 0000000..e2c4d83
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case7.sql
@@ -0,0 +1 @@
+select col1, key from testQueryCasesOnColumnPartitionedTable WHERE key BETWEEN 35 AND 48;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/f1f36ec5/tajo-core/src/test/resources/results/TestTablePartitions/case4.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case4.result b/tajo-core/src/test/resources/results/TestTablePartitions/case4.result
new file mode 100644
index 0000000..bcf2197
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case4.result
@@ -0,0 +1,7 @@
+?pow
+-------------------------------
+289.0
+1296.0
+1444.0
+2025.0
+2401.0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/f1f36ec5/tajo-core/src/test/resources/results/TestTablePartitions/case5.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case5.result b/tajo-core/src/test/resources/results/TestTablePartitions/case5.result
new file mode 100644
index 0000000..2c2918d
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case5.result
@@ -0,0 +1,7 @@
+?round
+-------------------------------
+324
+1369
+1521
+2116
+2500
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/f1f36ec5/tajo-core/src/test/resources/results/TestTablePartitions/case6.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case6.result b/tajo-core/src/test/resources/results/TestTablePartitions/case6.result
new file mode 100644
index 0000000..3c49070
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case6.result
@@ -0,0 +1,7 @@
+col1,key
+-------------------------------
+3,49.0
+3,45.0
+2,38.0
+1,36.0
+1,17.0
\ No newline at end of file
[7/9] git commit: TAJO-737: Change version message when daemon starts
up. (hyunsik)
Posted by hy...@apache.org.
TAJO-737: Change version message when daemon starts up. (hyunsik)
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/bbf2461a
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/bbf2461a
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/bbf2461a
Branch: refs/heads/window_function
Commit: bbf2461a5bddc77b3597030976158e00ee88fe0d
Parents: c367377
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Apr 21 16:38:48 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Apr 21 16:39:15 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 2 +
pom.xml | 1 +
tajo-common/pom.xml | 45 +++
.../java/org/apache/tajo/util/StringUtils.java | 68 ++++
.../java/org/apache/tajo/util/VersionInfo.java | 177 ++++++++++
.../main/resources/tajo-version-info.properties | 26 ++
.../java/org/apache/tajo/master/TajoMaster.java | 2 +-
.../java/org/apache/tajo/worker/TajoWorker.java | 2 +-
tajo-maven-plugins/pom.xml | 88 +++++
.../tajo/maven/plugin/protoc/ProtocMojo.java | 114 ++++++
.../org/apache/tajo/maven/plugin/util/Exec.java | 117 +++++++
.../tajo/maven/plugin/util/FileSetUtils.java | 61 ++++
.../plugin/versioninfo/VersionInfoMojo.java | 344 +++++++++++++++++++
13 files changed, 1045 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7f22b06..7f1b175 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -153,6 +153,8 @@ Release 0.8.0 - unreleased
IMPROVEMENTS
+ TAJO-737: Change version message when daemon starts up. (hyunsik)
+
TAJO-768: Improve the log4j configuration. (hyoungjunkim via jinho)
TAJO-755: ALTER TABLESPACE LOCATION support. (hyunsilk)
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 964d984..6eea5c8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -78,6 +78,7 @@
</properties>
<modules>
+ <module>tajo-maven-plugins</module>
<module>tajo-project</module>
<module>tajo-common</module>
<module>tajo-algebra</module>
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-common/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-common/pom.xml b/tajo-common/pom.xml
index 35ac3e5..f334380 100644
--- a/tajo-common/pom.xml
+++ b/tajo-common/pom.xml
@@ -46,8 +46,53 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
</repositories>
<build>
+ <!--
+ Include all files in src/main/resources. By default, do not apply property
+ substitution (filtering=false), but do apply property substitution to
+ tajo-version-info.properties (filtering=true). This will substitute the
+ version information correctly, but prevent Maven from altering other files
+ like tajo-default.xml.
+ -->
+ <resources>
+ <resource>
+ <directory>${basedir}/src/main/resources</directory>
+ <excludes>
+ <exclude>tajo-version-info.properties</exclude>
+ </excludes>
+ <filtering>false</filtering>
+ </resource>
+ <resource>
+ <directory>${basedir}/src/main/resources</directory>
+ <includes>
+ <include>tajo-version-info.properties</include>
+ </includes>
+ <filtering>true</filtering>
+ </resource>
+ </resources>
<plugins>
<plugin>
+ <groupId>org.apache.tajo</groupId>
+ <artifactId>tajo-maven-plugins</artifactId>
+ <executions>
+ <execution>
+ <id>version-info</id>
+ <phase>generate-resources</phase>
+ <goals>
+ <goal>version-info</goal>
+ </goals>
+ <configuration>
+ <source>
+ <directory>${basedir}/src/main</directory>
+ <includes>
+ <include>java/**/*.java</include>
+ <include>proto/**/*.proto</include>
+ </includes>
+ </source>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
index 32edc31..ed9014d 100644
--- a/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
+++ b/tajo-common/src/main/java/org/apache/tajo/util/StringUtils.java
@@ -18,9 +18,20 @@
package org.apache.tajo.util;
+import org.apache.commons.lang.SystemUtils;
+import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.SignalLogger;
+
+import java.util.Arrays;
+
public class StringUtils {
/**
+ * Priority of the StringUtils shutdown hook.
+ */
+ public static final int SHUTDOWN_HOOK_PRIORITY = 0;
+
+ /**
*
* Given the time in long milliseconds, returns a
* String in the format X hrs, Y mins, S sec, M msecs
@@ -97,4 +108,61 @@ public class StringUtils {
public static String escapeLike(String literal) {
return literal.replaceAll(LIKE_SPECIAL_CHARACTERS, "\\\\$1");
}
+
+ /**
+ * Return a message for logging.
+ * @param prefix prefix keyword for the message
+ * @param msg content of the message
+ * @return a message for logging
+ */
+ private static String toStartupShutdownString(String prefix, String [] msg) {
+ StringBuilder b = new StringBuilder(prefix);
+ b.append("\n/************************************************************");
+ for(String s : msg)
+ b.append("\n" + prefix + s);
+ b.append("\n************************************************************/");
+ return b.toString();
+ }
+
+ /**
+ * Print a log message for starting up and shutting down
+ * @param clazz the class of the server
+ * @param args arguments
+ * @param LOG the target log object
+ */
+ public static void startupShutdownMessage(Class<?> clazz, String[] args,
+ final org.apache.commons.logging.Log LOG) {
+ final String hostname = org.apache.hadoop.net.NetUtils.getHostname();
+ final String classname = clazz.getSimpleName();
+ LOG.info(
+ toStartupShutdownString("STARTUP_MSG: ", new String[] {
+ "Starting " + classname,
+ " host = " + hostname,
+ " args = " + Arrays.asList(args),
+ " version = " + org.apache.tajo.util.VersionInfo.getVersion(),
+ " classpath = " + System.getProperty("java.class.path"),
+ " build = " + org.apache.tajo.util.VersionInfo.getUrl() + " -r "
+ + org.apache.tajo.util.VersionInfo.getRevision()
+ + "; compiled by '" + org.apache.tajo.util.VersionInfo.getUser()
+ + "' on " + org.apache.tajo.util.VersionInfo.getDate(),
+ " java = " + System.getProperty("java.version") }
+ )
+ );
+
+ if (SystemUtils.IS_OS_UNIX) {
+ try {
+ SignalLogger.INSTANCE.register(LOG);
+ } catch (Throwable t) {
+ LOG.warn("failed to register any UNIX signal loggers: ", t);
+ }
+ }
+ ShutdownHookManager.get().addShutdownHook(
+ new Runnable() {
+ @Override
+ public void run() {
+ LOG.info(toStartupShutdownString("SHUTDOWN_MSG: ", new String[]{
+ "Shutting down " + classname + " at " + hostname}));
+ }
+ }, SHUTDOWN_HOOK_PRIORITY);
+ }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-common/src/main/java/org/apache/tajo/util/VersionInfo.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/util/VersionInfo.java b/tajo-common/src/main/java/org/apache/tajo/util/VersionInfo.java
new file mode 100644
index 0000000..39e0f54
--- /dev/null
+++ b/tajo-common/src/main/java/org/apache/tajo/util/VersionInfo.java
@@ -0,0 +1,177 @@
+package org.apache.tajo.util;
+
+/*
+ * 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.
+ */
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.ClassUtil;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
+/**
+ * This class returns build information about Hadoop components.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class VersionInfo {
+ private static final Log LOG = LogFactory.getLog(VersionInfo.class);
+
+ private Properties info;
+
+ protected VersionInfo(String component) {
+ info = new Properties();
+ String versionInfoFile = component + "-version-info.properties";
+ try {
+ InputStream is = Thread.currentThread().getContextClassLoader()
+ .getResourceAsStream(versionInfoFile);
+ if (is == null) {
+ throw new IOException("Resource not found");
+ }
+ info.load(is);
+ } catch (IOException ex) {
+ LogFactory.getLog(getClass()).warn("Could not read '" +
+ versionInfoFile + "', " + ex.toString(), ex);
+ }
+ }
+
+ protected String _getVersion() {
+ return info.getProperty("version", "Unknown");
+ }
+
+ protected String _getRevision() {
+ return info.getProperty("revision", "Unknown");
+ }
+
+ protected String _getBranch() {
+ return info.getProperty("branch", "Unknown");
+ }
+
+ protected String _getDate() {
+ return info.getProperty("date", "Unknown");
+ }
+
+ protected String _getUser() {
+ return info.getProperty("user", "Unknown");
+ }
+
+ protected String _getUrl() {
+ return info.getProperty("url", "Unknown");
+ }
+
+ protected String _getSrcChecksum() {
+ return info.getProperty("srcChecksum", "Unknown");
+ }
+
+ protected String _getBuildVersion(){
+ return getVersion() +
+ " from " + _getRevision() +
+ " by " + _getUser() +
+ " source checksum " + _getSrcChecksum();
+ }
+
+ protected String _getProtocVersion() {
+ return info.getProperty("protocVersion", "Unknown");
+ }
+
+ private static VersionInfo TAJO_VERSION_INFO = new VersionInfo("tajo");
+ /**
+ * Get the Hadoop version.
+ * @return the Hadoop version string, eg. "0.6.3-dev"
+ */
+ public static String getVersion() {
+ return TAJO_VERSION_INFO._getVersion();
+ }
+
+ /**
+ * Get the subversion revision number for the root directory
+ * @return the revision number, eg. "451451"
+ */
+ public static String getRevision() {
+ return TAJO_VERSION_INFO._getRevision();
+ }
+
+ /**
+ * Get the branch on which this originated.
+ * @return The branch name, e.g. "trunk" or "branches/branch-0.20"
+ */
+ public static String getBranch() {
+ return TAJO_VERSION_INFO._getBranch();
+ }
+
+ /**
+ * The date that Hadoop was compiled.
+ * @return the compilation date in unix date format
+ */
+ public static String getDate() {
+ return TAJO_VERSION_INFO._getDate();
+ }
+
+ /**
+ * The user that compiled Hadoop.
+ * @return the username of the user
+ */
+ public static String getUser() {
+ return TAJO_VERSION_INFO._getUser();
+ }
+
+ /**
+ * Get the subversion URL for the root Hadoop directory.
+ */
+ public static String getUrl() {
+ return TAJO_VERSION_INFO._getUrl();
+ }
+
+ /**
+ * Get the checksum of the source files from which Hadoop was
+ * built.
+ **/
+ public static String getSrcChecksum() {
+ return TAJO_VERSION_INFO._getSrcChecksum();
+ }
+
+ /**
+ * Returns the buildVersion which includes version,
+ * revision, user and date.
+ */
+ public static String getBuildVersion(){
+ return TAJO_VERSION_INFO._getBuildVersion();
+ }
+
+ /**
+ * Returns the protoc version used for the build.
+ */
+ public static String getProtocVersion(){
+ return TAJO_VERSION_INFO._getProtocVersion();
+ }
+
+ public static void main(String[] args) {
+ LOG.debug("version: "+ getVersion());
+ System.out.println("Tajo " + getVersion());
+ System.out.println("Git " + getUrl() + " -r " + getRevision());
+ System.out.println("Compiled by " + getUser() + " on " + getDate());
+ System.out.println("Compiled with protoc " + getProtocVersion());
+ System.out.println("From source with checksum " + getSrcChecksum());
+ System.out.println("This command was run using " + ClassUtil.findContainingJar(VersionInfo.class));
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-common/src/main/resources/tajo-version-info.properties
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/resources/tajo-version-info.properties b/tajo-common/src/main/resources/tajo-version-info.properties
new file mode 100644
index 0000000..ad9a24d
--- /dev/null
+++ b/tajo-common/src/main/resources/tajo-version-info.properties
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+version=${pom.version}
+revision=${version-info.scm.commit}
+branch=${version-info.scm.branch}
+user=${user.name}
+date=${version-info.build.time}
+url=${version-info.scm.uri}
+srcChecksum=${version-info.source.md5}
+protocVersion=${protobuf.version}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
index 9d54bb5..dfae300 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -58,6 +57,7 @@ import org.apache.tajo.storage.StorageManagerFactory;
import org.apache.tajo.util.ClassUtil;
import org.apache.tajo.util.CommonTestingUtil;
import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.StringUtils;
import org.apache.tajo.util.metrics.TajoSystemMetrics;
import org.apache.tajo.webapp.QueryExecutorServlet;
import org.apache.tajo.webapp.StaticHttpServer;
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
index 0b8d6c2..3768edf 100644
--- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
+++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.shell.PathData;
import org.apache.hadoop.service.CompositeService;
-import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.tajo.QueryId;
import org.apache.tajo.TajoConstants;
@@ -45,6 +44,7 @@ import org.apache.tajo.rpc.RpcConnectionPool;
import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos;
import org.apache.tajo.util.CommonTestingUtil;
import org.apache.tajo.util.NetUtils;
+import org.apache.tajo.util.StringUtils;
import org.apache.tajo.util.TajoIdUtils;
import org.apache.tajo.util.metrics.TajoSystemMetrics;
import org.apache.tajo.webapp.StaticHttpServer;
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-maven-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/tajo-maven-plugins/pom.xml b/tajo-maven-plugins/pom.xml
new file mode 100644
index 0000000..7de67ef
--- /dev/null
+++ b/tajo-maven-plugins/pom.xml
@@ -0,0 +1,88 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.tajo</groupId>
+ <artifactId>tajo-project</artifactId>
+ <version>0.8.0-SNAPSHOT</version>
+ <relativePath>../tajo-project</relativePath>
+ </parent>
+ <groupId>org.apache.tajo</groupId>
+ <artifactId>tajo-maven-plugins</artifactId>
+ <packaging>maven-plugin</packaging>
+ <name>Tajo Maven Plugins</name>
+ <properties>
+ <maven.dependency.version>3.0</maven.dependency.version>
+ </properties>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-plugin-api</artifactId>
+ <version>${maven.dependency.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven</groupId>
+ <artifactId>maven-core</artifactId>
+ <version>${maven.dependency.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.plugin-tools</groupId>
+ <artifactId>maven-plugin-annotations</artifactId>
+ <version>${maven.dependency.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>3.8.1</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-plugin-plugin</artifactId>
+ <version>${maven.dependency.version}</version>
+ <configuration>
+ <skipErrorNoDescriptorsFound>true</skipErrorNoDescriptorsFound>
+ </configuration>
+ <executions>
+ <execution>
+ <id>mojo-descriptor</id>
+ <goals>
+ <goal>descriptor</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <!--
+ Skip Clover instrumentation for this module to prevent error finding Clover
+ classes during plugin execution when running a build with Clover enabled.
+ -->
+ <plugin>
+ <groupId>com.atlassian.maven.plugins</groupId>
+ <artifactId>maven-clover2-plugin</artifactId>
+ <version>3.0.5</version>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/protoc/ProtocMojo.java
----------------------------------------------------------------------
diff --git a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/protoc/ProtocMojo.java b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/protoc/ProtocMojo.java
new file mode 100644
index 0000000..498f85d
--- /dev/null
+++ b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/protoc/ProtocMojo.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2012 The Apache Software Foundation.
+ *
+ * Licensed 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.maven.plugin.protoc;
+
+import org.apache.tajo.maven.plugin.util.Exec;
+import org.apache.tajo.maven.plugin.util.FileSetUtils;
+import org.apache.maven.model.FileSet;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugins.annotations.LifecyclePhase;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+import org.apache.maven.project.MavenProject;
+import org.apache.tajo.maven.plugin.util.Exec;
+import org.apache.tajo.maven.plugin.util.FileSetUtils;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+
+
+@Mojo(name="protoc", defaultPhase = LifecyclePhase.GENERATE_SOURCES)
+public class ProtocMojo extends AbstractMojo {
+
+ @Parameter(defaultValue="${project}")
+ private MavenProject project;
+
+ @Parameter
+ private File[] imports;
+
+ @Parameter(defaultValue="${project.build.directory}/generated-sources/java")
+ private File output;
+
+ @Parameter(required=true)
+ private FileSet source;
+
+ @Parameter
+ private String protocCommand;
+
+ @Parameter(required=true)
+ private String protocVersion;
+
+ public void execute() throws MojoExecutionException {
+ try {
+ if (protocCommand == null || protocCommand.trim().isEmpty()) {
+ protocCommand = "protoc";
+ }
+ List<String> command = new ArrayList<String>();
+ command.add(protocCommand);
+ command.add("--version");
+ Exec exec = new Exec(this);
+ List<String> out = new ArrayList<String>();
+ if (exec.run(command, out) == 127) {
+ getLog().error("protoc, not found at: " + protocCommand);
+ throw new MojoExecutionException("protoc failure");
+ } else {
+ if (out.isEmpty()) {
+ getLog().error("stdout: " + out);
+ throw new MojoExecutionException(
+ "'protoc --version' did not return a version");
+ } else {
+ if (!out.get(0).endsWith(protocVersion)) {
+ throw new MojoExecutionException(
+ "protoc version is '" + out.get(0) + "', expected version is '"
+ + protocVersion + "'");
+ }
+ }
+ }
+ if (!output.mkdirs()) {
+ if (!output.exists()) {
+ throw new MojoExecutionException("Could not create directory: " +
+ output);
+ }
+ }
+ command = new ArrayList<String>();
+ command.add(protocCommand);
+ command.add("--java_out=" + output.getCanonicalPath());
+ if (imports != null) {
+ for (File i : imports) {
+ command.add("-I" + i.getCanonicalPath());
+ }
+ }
+ for (File f : FileSetUtils.convertFileSetToFiles(source)) {
+ command.add(f.getCanonicalPath());
+ }
+ exec = new Exec(this);
+ out = new ArrayList<String>();
+ if (exec.run(command, out) != 0) {
+ getLog().error("protoc compiler error");
+ for (String s : out) {
+ getLog().error(s);
+ }
+ throw new MojoExecutionException("protoc failure");
+ }
+ } catch (Throwable ex) {
+ throw new MojoExecutionException(ex.toString(), ex);
+ }
+ project.addCompileSourceRoot(output.getAbsolutePath());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/Exec.java
----------------------------------------------------------------------
diff --git a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/Exec.java b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/Exec.java
new file mode 100644
index 0000000..89ba6fb
--- /dev/null
+++ b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/Exec.java
@@ -0,0 +1,117 @@
+/*
+ * Copyright 2012 The Apache Software Foundation.
+ *
+ * Licensed 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.maven.plugin.util;
+
+import org.apache.maven.plugin.Mojo;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Exec is a helper class for executing an external process from a mojo.
+ */
+public class Exec {
+ private Mojo mojo;
+
+ /**
+ * Creates a new Exec instance for executing an external process from the given
+ * mojo.
+ *
+ * @param mojo Mojo executing external process
+ */
+ public Exec(Mojo mojo) {
+ this.mojo = mojo;
+ }
+
+ /**
+ * Runs the specified command and saves each line of the command's output to
+ * the given list.
+ *
+ * @param command List<String> containing command and all arguments
+ * @param output List<String> in/out parameter to receive command output
+ * @return int exit code of command
+ */
+ public int run(List<String> command, List<String> output) {
+ int retCode = 1;
+ ProcessBuilder pb = new ProcessBuilder(command);
+ try {
+ Process p = pb.start();
+ OutputBufferThread stdOut = new OutputBufferThread(p.getInputStream());
+ OutputBufferThread stdErr = new OutputBufferThread(p.getErrorStream());
+ stdOut.start();
+ stdErr.start();
+ retCode = p.waitFor();
+ if (retCode != 0) {
+ mojo.getLog().warn(command + " failed with error code " + retCode);
+ for (String s : stdErr.getOutput()) {
+ mojo.getLog().debug(s);
+ }
+ }
+ stdOut.join();
+ stdErr.join();
+ output.addAll(stdOut.getOutput());
+ } catch (Exception ex) {
+ mojo.getLog().warn(command + " failed: " + ex.toString());
+ }
+ return retCode;
+ }
+
+ /**
+ * OutputBufferThread is a background thread for consuming and storing output
+ * of the external process.
+ */
+ private static class OutputBufferThread extends Thread {
+ private List<String> output;
+ private BufferedReader reader;
+
+ /**
+ * Creates a new OutputBufferThread to consume the given InputStream.
+ *
+ * @param is InputStream to consume
+ */
+ public OutputBufferThread(InputStream is) {
+ this.setDaemon(true);
+ output = new ArrayList<String>();
+ reader = new BufferedReader(new InputStreamReader(is));
+ }
+
+ @Override
+ public void run() {
+ try {
+ String line = reader.readLine();
+ while (line != null) {
+ output.add(line);
+ line = reader.readLine();
+ }
+ } catch (IOException ex) {
+ throw new RuntimeException("make failed with error code " + ex.toString());
+ }
+ }
+
+ /**
+ * Returns every line consumed from the input.
+ *
+ * @return List<String> every line consumed from the input
+ */
+ public List<String> getOutput() {
+ return output;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/FileSetUtils.java
----------------------------------------------------------------------
diff --git a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/FileSetUtils.java b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/FileSetUtils.java
new file mode 100644
index 0000000..f034f44
--- /dev/null
+++ b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/util/FileSetUtils.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2012 The Apache Software Foundation.
+ *
+ * Licensed 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.maven.plugin.util;
+
+import org.apache.maven.model.FileSet;
+import org.codehaus.plexus.util.FileUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * FileSetUtils contains helper methods for mojo implementations that need to
+ * work with a Maven FileSet.
+ */
+public class FileSetUtils {
+
+ /**
+ * Returns a string containing every element of the given list, with each
+ * element separated by a comma.
+ *
+ * @param list List of all elements
+ * @return String containing every element, comma-separated
+ */
+ private static String getCommaSeparatedList(List list) {
+ StringBuilder buffer = new StringBuilder();
+ String separator = "";
+ for (Object e : list) {
+ buffer.append(separator).append(e);
+ separator = ",";
+ }
+ return buffer.toString();
+ }
+
+ /**
+ * Converts a Maven FileSet to a list of File objects.
+ *
+ * @param source FileSet to convert
+ * @return List<File> containing every element of the FileSet as a File
+ * @throws IOException if an I/O error occurs while trying to find the files
+ */
+ @SuppressWarnings("unchecked")
+ public static List<File> convertFileSetToFiles(FileSet source) throws IOException {
+ String includes = getCommaSeparatedList(source.getIncludes());
+ String excludes = getCommaSeparatedList(source.getExcludes());
+ return FileUtils.getFiles(new File(source.getDirectory()), includes, excludes);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/bbf2461a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/versioninfo/VersionInfoMojo.java
----------------------------------------------------------------------
diff --git a/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/versioninfo/VersionInfoMojo.java b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/versioninfo/VersionInfoMojo.java
new file mode 100644
index 0000000..5347f40
--- /dev/null
+++ b/tajo-maven-plugins/src/main/java/org/apache/tajo/maven/plugin/versioninfo/VersionInfoMojo.java
@@ -0,0 +1,344 @@
+/*
+ * Copyright 2012 The Apache Software Foundation.
+ *
+ * Licensed 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.maven.plugin.versioninfo;
+
+import org.apache.tajo.maven.plugin.util.Exec;
+import org.apache.tajo.maven.plugin.util.FileSetUtils;
+import org.apache.maven.model.FileSet;
+import org.apache.maven.plugin.AbstractMojo;
+import org.apache.maven.plugin.MojoExecutionException;
+import org.apache.maven.plugins.annotations.Mojo;
+import org.apache.maven.plugins.annotations.Parameter;
+import org.apache.maven.project.MavenProject;
+import org.apache.tajo.maven.plugin.util.Exec;
+import org.apache.tajo.maven.plugin.util.FileSetUtils;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.List;
+import java.util.TimeZone;
+
+/**
+ * VersionInfoMojo calculates information about the current version of the
+ * codebase and exports the information as properties for further use in a Maven
+ * build. The version information includes build time, SCM URI, SCM branch, SCM
+ * commit, and an MD5 checksum of the contents of the files in the codebase.
+ */
+@Mojo(name="version-info")
+public class VersionInfoMojo extends AbstractMojo {
+
+ @Parameter(defaultValue="${project}")
+ private MavenProject project;
+
+ @Parameter(required=true)
+ private FileSet source;
+
+ @Parameter(defaultValue="version-info.build.time")
+ private String buildTimeProperty;
+
+ @Parameter(defaultValue="version-info.source.md5")
+ private String md5Property;
+
+ @Parameter(defaultValue="version-info.scm.uri")
+ private String scmUriProperty;
+
+ @Parameter(defaultValue="version-info.scm.branch")
+ private String scmBranchProperty;
+
+ @Parameter(defaultValue="version-info.scm.commit")
+ private String scmCommitProperty;
+
+ @Parameter(defaultValue="git")
+ private String gitCommand;
+
+ @Parameter(defaultValue="svn")
+ private String svnCommand;
+
+ private enum SCM {NONE, SVN, GIT}
+
+ @Override
+ public void execute() throws MojoExecutionException {
+ try {
+ SCM scm = determineSCM();
+ project.getProperties().setProperty(buildTimeProperty, getBuildTime());
+ project.getProperties().setProperty(scmUriProperty, getSCMUri(scm));
+ project.getProperties().setProperty(scmBranchProperty, getSCMBranch(scm));
+ project.getProperties().setProperty(scmCommitProperty, getSCMCommit(scm));
+ project.getProperties().setProperty(md5Property, computeMD5());
+ } catch (Throwable ex) {
+ throw new MojoExecutionException(ex.toString(), ex);
+ }
+ }
+
+ /**
+ * Returns a string representing current build time.
+ *
+ * @return String representing current build time
+ */
+ private String getBuildTime() {
+ DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm'Z'");
+ dateFormat.setTimeZone(TimeZone.getTimeZone("UTC"));
+ return dateFormat.format(new Date());
+ }
+ private List<String> scmOut;
+
+ /**
+ * Determines which SCM is in use (Subversion, git, or none) and captures
+ * output of the SCM command for later parsing.
+ *
+ * @return SCM in use for this build
+ * @throws Exception if any error occurs attempting to determine SCM
+ */
+ private SCM determineSCM() throws Exception {
+ Exec exec = new Exec(this);
+ SCM scm = SCM.NONE;
+ scmOut = new ArrayList<String>();
+ int ret = exec.run(Arrays.asList(svnCommand, "info"), scmOut);
+ if (ret == 0) {
+ scm = SCM.SVN;
+ } else {
+ ret = exec.run(Arrays.asList(gitCommand, "branch"), scmOut);
+ if (ret == 0) {
+ ret = exec.run(Arrays.asList(gitCommand, "remote", "-v"), scmOut);
+ if (ret != 0) {
+ scm = SCM.NONE;
+ scmOut = null;
+ } else {
+ ret = exec.run(Arrays.asList(gitCommand, "log", "-n", "1"), scmOut);
+ if (ret != 0) {
+ scm = SCM.NONE;
+ scmOut = null;
+ } else {
+ scm = SCM.GIT;
+ }
+ }
+ }
+ }
+ if (scmOut != null) {
+ getLog().debug(scmOut.toString());
+ }
+ getLog().info("SCM: " + scm);
+ return scm;
+ }
+
+ /**
+ * Return URI and branch of Subversion repository.
+ *
+ * @param str String Subversion info output containing URI and branch
+ * @return String[] containing URI and branch
+ */
+ private String[] getSvnUriInfo(String str) {
+ String[] res = new String[]{"Unknown", "Unknown"};
+ try {
+ String path = str;
+ int index = path.indexOf("trunk");
+ if (index > -1) {
+ res[0] = path.substring(0, index - 1);
+ res[1] = "trunk";
+ } else {
+ index = path.indexOf("branches");
+ if (index > -1) {
+ res[0] = path.substring(0, index - 1);
+ int branchIndex = index + "branches".length() + 1;
+ index = path.indexOf("/", branchIndex);
+ if (index > -1) {
+ res[1] = path.substring(branchIndex, index);
+ } else {
+ res[1] = path.substring(branchIndex);
+ }
+ }
+ }
+ } catch (Exception ex) {
+ getLog().warn("Could not determine URI & branch from SVN URI: " + str);
+ }
+ return res;
+ }
+
+ /**
+ * Parses SCM output and returns URI of SCM.
+ *
+ * @param scm SCM in use for this build
+ * @return String URI of SCM
+ */
+ private String getSCMUri(SCM scm) {
+ String uri = "Unknown";
+ switch (scm) {
+ case SVN:
+ for (String s : scmOut) {
+ if (s.startsWith("URL:")) {
+ uri = s.substring(4).trim();
+ uri = getSvnUriInfo(uri)[0];
+ break;
+ }
+ }
+ break;
+ case GIT:
+ for (String s : scmOut) {
+ if (s.startsWith("origin") && s.endsWith("(fetch)")) {
+ uri = s.substring("origin".length());
+ uri = uri.substring(0, uri.length() - "(fetch)".length());
+ break;
+ }
+ }
+ break;
+ }
+ return uri.trim();
+ }
+
+ /**
+ * Parses SCM output and returns commit of SCM.
+ *
+ * @param scm SCM in use for this build
+ * @return String commit of SCM
+ */
+ private String getSCMCommit(SCM scm) {
+ String commit = "Unknown";
+ switch (scm) {
+ case SVN:
+ for (String s : scmOut) {
+ if (s.startsWith("Revision:")) {
+ commit = s.substring("Revision:".length());
+ break;
+ }
+ }
+ break;
+ case GIT:
+ for (String s : scmOut) {
+ if (s.startsWith("commit")) {
+ commit = s.substring("commit".length());
+ break;
+ }
+ }
+ break;
+ }
+ return commit.trim();
+ }
+
+ /**
+ * Parses SCM output and returns branch of SCM.
+ *
+ * @param scm SCM in use for this build
+ * @return String branch of SCM
+ */
+ private String getSCMBranch(SCM scm) {
+ String branch = "Unknown";
+ switch (scm) {
+ case SVN:
+ for (String s : scmOut) {
+ if (s.startsWith("URL:")) {
+ branch = s.substring(4).trim();
+ branch = getSvnUriInfo(branch)[1];
+ break;
+ }
+ }
+ break;
+ case GIT:
+ for (String s : scmOut) {
+ if (s.startsWith("*")) {
+ branch = s.substring("*".length());
+ break;
+ }
+ }
+ break;
+ }
+ return branch.trim();
+ }
+
+ /**
+ * Reads and returns the full contents of the specified file.
+ *
+ * @param file File to read
+ * @return byte[] containing full contents of file
+ * @throws IOException if there is an I/O error while reading the file
+ */
+ private byte[] readFile(File file) throws IOException {
+ RandomAccessFile raf = new RandomAccessFile(file, "r");
+ byte[] buffer = new byte[(int) raf.length()];
+ raf.readFully(buffer);
+ raf.close();
+ return buffer;
+ }
+
+ /**
+ * Given a list of files, computes and returns an MD5 checksum of the full
+ * contents of all files.
+ *
+ * @param files List<File> containing every file to input into the MD5 checksum
+ * @return byte[] calculated MD5 checksum
+ * @throws IOException if there is an I/O error while reading a file
+ * @throws NoSuchAlgorithmException if the MD5 algorithm is not supported
+ */
+ private byte[] computeMD5(List<File> files) throws IOException, NoSuchAlgorithmException {
+ MessageDigest md5 = MessageDigest.getInstance("MD5");
+ for (File file : files) {
+ getLog().debug("Computing MD5 for: " + file);
+ md5.update(readFile(file));
+ }
+ return md5.digest();
+ }
+
+ /**
+ * Converts bytes to a hexadecimal string representation and returns it.
+ *
+ * @param array byte[] to convert
+ * @return String containing hexadecimal representation of bytes
+ */
+ private String byteArrayToString(byte[] array) {
+ StringBuilder sb = new StringBuilder();
+ for (byte b : array) {
+ sb.append(Integer.toHexString(0xff & b));
+ }
+ return sb.toString();
+ }
+
+ /**
+ * Computes and returns an MD5 checksum of the contents of all files in the
+ * input Maven FileSet.
+ *
+ * @return String containing hexadecimal representation of MD5 checksum
+ * @throws Exception if there is any error while computing the MD5 checksum
+ */
+ private String computeMD5() throws Exception {
+ List<File> files = FileSetUtils.convertFileSetToFiles(source);
+ // File order of MD5 calculation is significant. Sorting is done on
+ // unix-format names, case-folded, in order to get a platform-independent
+ // sort and calculate the same MD5 on all platforms.
+ Collections.sort(files, new Comparator<File>() {
+ @Override
+ public int compare(File lhs, File rhs) {
+ return normalizePath(lhs).compareTo(normalizePath(rhs));
+ }
+
+ private String normalizePath(File file) {
+ return file.getPath().toUpperCase().replaceAll("\\\\", "/");
+ }
+ });
+ byte[] md5 = computeMD5(files);
+ String md5str = byteArrayToString(md5);
+ getLog().info("Computed MD5: " + md5str);
+ return md5str;
+ }
+}
[9/9] git commit: Merge branch 'master' of
https://git-wip-us.apache.org/repos/asf/tajo into window_function
Posted by hy...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/tajo into window_function
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/39d4f2d7
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/39d4f2d7
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/39d4f2d7
Branch: refs/heads/window_function
Commit: 39d4f2d7ba1fd16aadbc2809d6278f49a9857f72
Parents: 1b3ec37 cbe1d6e
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Apr 21 20:12:21 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Apr 21 20:12:21 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 14 +
pom.xml | 1 +
.../org/apache/tajo/catalog/DDLBuilder.java | 1 -
.../java/org/apache/tajo/client/TajoDump.java | 18 +-
tajo-common/pom.xml | 45 +++
.../java/org/apache/tajo/conf/TajoConf.java | 2 +-
.../java/org/apache/tajo/util/StringUtils.java | 68 ++++
.../org/apache/tajo/util/TimeStampUtil.java | 11 +-
.../java/org/apache/tajo/util/VersionInfo.java | 177 ++++++++++
.../main/resources/tajo-version-info.properties | 26 ++
.../org/apache/tajo/util/TestTimeStampUtil.java | 75 ++++
.../tajo/engine/eval/BasicEvalNodeVisitor.java | 4 +-
.../tajo/engine/eval/BetweenPredicateEval.java | 12 +
.../apache/tajo/engine/eval/CaseWhenEval.java | 14 +-
.../org/apache/tajo/engine/eval/CastEval.java | 4 +
.../apache/tajo/engine/eval/EvalTreeUtil.java | 68 +++-
.../apache/tajo/engine/eval/FunctionEval.java | 2 +-
.../org/apache/tajo/engine/eval/NotEval.java | 4 +
.../org/apache/tajo/engine/eval/SignedEval.java | 4 +
.../apache/tajo/engine/planner/LogicalPlan.java | 230 +++++++------
.../engine/planner/global/GlobalPlanner.java | 49 ++-
.../java/org/apache/tajo/master/TajoMaster.java | 2 +-
.../tajo/master/querymaster/SubQuery.java | 18 +
.../java/org/apache/tajo/worker/TajoWorker.java | 2 +-
.../org/apache/tajo/client/TestTajoDump.java | 46 +++
.../tajo/engine/query/TestJoinBroadcast.java | 14 +-
.../tajo/engine/query/TestTablePartitions.java | 50 +++
.../tajo/master/TestExecutionBlockCursor.java | 2 +-
.../querymaster/TestQueryUnitStatusUpdate.java | 8 +-
.../org/apache/tajo/client/TestTajoDump.java | 44 ---
.../resources/queries/TestNetTypes/testJoin.sql | 2 +-
.../queries/TestTablePartitions/case10.sql | 1 +
.../queries/TestTablePartitions/case11.sql | 1 +
.../queries/TestTablePartitions/case12.sql | 7 +
.../queries/TestTablePartitions/case13.sql | 11 +
.../queries/TestTablePartitions/case4.sql | 1 +
.../queries/TestTablePartitions/case5.sql | 1 +
.../queries/TestTablePartitions/case6.sql | 1 +
.../queries/TestTablePartitions/case7.sql | 1 +
.../queries/TestTablePartitions/case8.sql | 1 +
.../queries/TestTablePartitions/case9.sql | 1 +
.../testBroadcastSubquery2.result | 2 +-
.../results/TestTablePartitions/case10.result | 7 +
.../results/TestTablePartitions/case11.result | 7 +
.../results/TestTablePartitions/case12.result | 7 +
.../results/TestTablePartitions/case13.result | 7 +
.../results/TestTablePartitions/case4.result | 7 +
.../results/TestTablePartitions/case5.result | 7 +
.../results/TestTablePartitions/case6.result | 7 +
.../results/TestTablePartitions/case7.result | 5 +
.../results/TestTablePartitions/case8.result | 7 +
.../results/TestTablePartitions/case9.result | 7 +
.../results/TestTajoDump/testDump1.result | 9 +-
.../testBuildDDLForBaseTable.result | 1 -
.../testBuildDDLQuotedTableName2.result | 1 -
tajo-maven-plugins/pom.xml | 88 +++++
.../tajo/maven/plugin/protoc/ProtocMojo.java | 114 ++++++
.../org/apache/tajo/maven/plugin/util/Exec.java | 117 +++++++
.../tajo/maven/plugin/util/FileSetUtils.java | 61 ++++
.../plugin/versioninfo/VersionInfoMojo.java | 344 +++++++++++++++++++
.../org/apache/tajo/rpc/NettyServerBase.java | 3 +
.../apache/tajo/rpc/ProtoPipelineFactory.java | 9 +-
62 files changed, 1654 insertions(+), 206 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/39d4f2d7/CHANGES.txt
----------------------------------------------------------------------
[6/9] git commit: TAJO-675: maximum frame size of frameDecoder should
be increased. (jinho)
Posted by hy...@apache.org.
TAJO-675: maximum frame size of frameDecoder should be increased. (jinho)
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/c367377f
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/c367377f
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/c367377f
Branch: refs/heads/window_function
Commit: c367377fa2887458fd246b7481cfc49865dc5ff9
Parents: 2b27f7d
Author: jinossy <ji...@gmail.com>
Authored: Mon Apr 21 16:21:48 2014 +0900
Committer: jinossy <ji...@gmail.com>
Committed: Mon Apr 21 16:21:48 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 2 ++
.../main/java/org/apache/tajo/rpc/ProtoPipelineFactory.java | 9 ++++-----
2 files changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/c367377f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0fcc83d..7f22b06 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -334,6 +334,8 @@ Release 0.8.0 - unreleased
BUG FIXES
+ TAJO-675: maximum frame size of frameDecoder should be increased. (jinho)
+
TAJO-748: Shuffle output numbers of join may be inconsistent. (jaehwa)
TAJO-777: Partition column in function parameter occurs NPE.
http://git-wip-us.apache.org/repos/asf/tajo/blob/c367377f/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoPipelineFactory.java
----------------------------------------------------------------------
diff --git a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoPipelineFactory.java b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoPipelineFactory.java
index b2a2004..7aa03e7 100644
--- a/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoPipelineFactory.java
+++ b/tajo-rpc/src/main/java/org/apache/tajo/rpc/ProtoPipelineFactory.java
@@ -23,10 +23,10 @@ import org.jboss.netty.channel.ChannelPipeline;
import org.jboss.netty.channel.ChannelPipelineFactory;
import org.jboss.netty.channel.ChannelUpstreamHandler;
import org.jboss.netty.channel.Channels;
-import org.jboss.netty.handler.codec.frame.LengthFieldBasedFrameDecoder;
-import org.jboss.netty.handler.codec.frame.LengthFieldPrepender;
import org.jboss.netty.handler.codec.protobuf.ProtobufDecoder;
import org.jboss.netty.handler.codec.protobuf.ProtobufEncoder;
+import org.jboss.netty.handler.codec.protobuf.ProtobufVarint32FrameDecoder;
+import org.jboss.netty.handler.codec.protobuf.ProtobufVarint32LengthFieldPrepender;
public class ProtoPipelineFactory implements ChannelPipelineFactory {
private final ChannelUpstreamHandler handler;
@@ -40,10 +40,9 @@ public class ProtoPipelineFactory implements ChannelPipelineFactory {
public ChannelPipeline getPipeline() throws Exception {
ChannelPipeline p = Channels.pipeline();
- p.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(1048576*2, 0, 4,
- 0, 4));
+ p.addLast("frameDecoder", new ProtobufVarint32FrameDecoder());
p.addLast("protobufDecoder", new ProtobufDecoder(defaultInstance));
- p.addLast("frameEncoder", new LengthFieldPrepender(4));
+ p.addLast("frameEncoder", new ProtobufVarint32LengthFieldPrepender());
p.addLast("protobufEncoder", new ProtobufEncoder());
p.addLast("handler", handler);
return p;
[3/9] git commit: TAJO-777: Partition column in function parameter
occurs NPE. (Hyoungjun Kim via hyunsik)
Posted by hy...@apache.org.
TAJO-777: Partition column in function parameter occurs NPE. (Hyoungjun Kim via hyunsik)
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/e56a7a41
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/e56a7a41
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/e56a7a41
Branch: refs/heads/window_function
Commit: e56a7a41fb6b27e093c5c49a2d86a8cabc203156
Parents: 6a005ce
Author: Hyunsik Choi <hy...@apache.org>
Authored: Mon Apr 21 11:36:23 2014 +0900
Committer: Hyunsik Choi <hy...@apache.org>
Committed: Mon Apr 21 11:36:23 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 3 +
.../org/apache/tajo/catalog/DDLBuilder.java | 1 -
.../java/org/apache/tajo/client/TajoDump.java | 18 +++---
.../tajo/engine/eval/BasicEvalNodeVisitor.java | 4 +-
.../tajo/engine/eval/BetweenPredicateEval.java | 12 ++++
.../apache/tajo/engine/eval/CaseWhenEval.java | 14 +++-
.../org/apache/tajo/engine/eval/CastEval.java | 4 ++
.../apache/tajo/engine/eval/EvalTreeUtil.java | 68 ++++++++++++++++++--
.../apache/tajo/engine/eval/FunctionEval.java | 2 +-
.../org/apache/tajo/engine/eval/NotEval.java | 4 ++
.../org/apache/tajo/engine/eval/SignedEval.java | 4 ++
.../org/apache/tajo/client/TestTajoDump.java | 46 +++++++++++++
.../tajo/engine/query/TestTablePartitions.java | 35 ++++++++++
.../org/apache/tajo/client/TestTajoDump.java | 44 -------------
.../queries/TestTablePartitions/case10.sql | 1 +
.../queries/TestTablePartitions/case8.sql | 1 +
.../queries/TestTablePartitions/case9.sql | 1 +
.../results/TestTablePartitions/case10.result | 7 ++
.../results/TestTablePartitions/case7.result | 5 ++
.../results/TestTablePartitions/case8.result | 7 ++
.../results/TestTablePartitions/case9.result | 7 ++
.../results/TestTajoDump/testDump1.result | 9 ++-
.../testBuildDDLForBaseTable.result | 1 -
.../testBuildDDLQuotedTableName2.result | 1 -
24 files changed, 228 insertions(+), 71 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 94b6a84..d6ed95a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -334,6 +334,9 @@ Release 0.8.0 - unreleased
BUG FIXES
+ TAJO-777: Partition column in function parameter occurs NPE.
+ (Hyoungjun Kim via hyunsik)
+
TAJO-763: Out of range problem in utc_usec_to(). (Ilhyun Suh via hyunsik)
TAJO-741: GreedyHeuristicJoinOrderAlgorithm removes some join pairs. (jaehwa)
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
----------------------------------------------------------------------
diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
index 1b76f16..77dccd2 100644
--- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
+++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/DDLBuilder.java
@@ -54,7 +54,6 @@ public class DDLBuilder {
sb.append("--\n")
.append("-- Name: ").append(CatalogUtil.denormalizeIdentifier(desc.getName())).append("; Type: TABLE;")
.append(" Storage: ").append(desc.getMeta().getStoreType().name());
- sb.append("\n-- Path: ").append(desc.getPath());
sb.append("\n--\n");
sb.append("CREATE TABLE ").append(CatalogUtil.denormalizeIdentifier(desc.getName()));
buildSchema(sb, desc.getSchema());
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
index 52711c9..60b1a7e 100644
--- a/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
+++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoDump.java
@@ -110,15 +110,15 @@ public class TajoDump {
}
PrintWriter writer = new PrintWriter(System.out);
- dump(client, userInfo, baseDatabaseName, isDumpingAllDatabases, true, writer);
+ dump(client, userInfo, baseDatabaseName, isDumpingAllDatabases, true, true, writer);
System.exit(0);
}
public static void dump(TajoClient client, UserGroupInformation userInfo, String baseDatabaseName,
- boolean isDumpingAllDatabases, boolean includeDate, PrintWriter out)
+ boolean isDumpingAllDatabases, boolean includeUserName, boolean includeDate, PrintWriter out)
throws SQLException, ServiceException {
- printHeader(out, userInfo, includeDate);
+ printHeader(out, userInfo, includeUserName, includeDate);
if (isDumpingAllDatabases) {
// sort database names in an ascending lexicographic order of the names.
@@ -134,13 +134,15 @@ public class TajoDump {
out.flush();
}
- private static void printHeader(PrintWriter writer, UserGroupInformation userInfo, boolean includeDate) {
+ private static void printHeader(PrintWriter writer, UserGroupInformation userInfo, boolean includeUSerName,
+ boolean includeDate) {
writer.write("--\n");
writer.write("-- Tajo database dump\n");
- writer.write("--\n");
- writer.write("-- Dump user: " + userInfo.getUserName() + "\n");
+ if (includeUSerName) {
+ writer.write("--\nDump user: " + userInfo.getUserName() + "\n");
+ }
if (includeDate) {
- writer.write("-- Dump date: " + toDateString() + "\n");
+ writer.write("\n-- Dump date: " + toDateString() + "\n");
}
writer.write("--\n");
writer.write("\n");
@@ -154,7 +156,7 @@ public class TajoDump {
writer.write("--\n");
writer.write("\n");
writer.write(String.format("CREATE DATABASE IF NOT EXISTS %s;", CatalogUtil.denormalizeIdentifier(databaseName)));
- writer.write("\n");
+ writer.write("\n\n");
// returned list is immutable.
List<String> tableNames = TUtil.newList(client.getTableList(databaseName));
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java
index 2208119..015e9db 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BasicEvalNodeVisitor.java
@@ -280,8 +280,8 @@ public class BasicEvalNodeVisitor<CONTEXT, RESULT> implements EvalNodeVisitor2<C
public RESULT visitIfThen(CONTEXT context, CaseWhenEval.IfThenEval evalNode, Stack<EvalNode> stack) {
RESULT result;
stack.push(evalNode);
- result = visitChild(context, evalNode.getConditionExpr(), stack);
- visitChild(context, evalNode.getResultExpr(), stack);
+ result = visitChild(context, evalNode.getCondition(), stack);
+ visitChild(context, evalNode.getResult(), stack);
stack.pop();
return result;
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java
index 0f30ac9..80e6494 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/BetweenPredicateEval.java
@@ -46,14 +46,26 @@ public class BetweenPredicateEval extends EvalNode {
this.end = end;
}
+ public void setPredicand(EvalNode predicand) {
+ this.predicand = predicand;
+ }
+
public EvalNode getPredicand() {
return predicand;
}
+ public void setBegin(EvalNode begin) {
+ this.begin = begin;
+ }
+
public EvalNode getBegin() {
return begin;
}
+ public void setEnd(EvalNode end) {
+ this.end = end;
+ }
+
public EvalNode getEnd() {
return end;
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java
index d08bfd3..0883e5c 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CaseWhenEval.java
@@ -63,7 +63,7 @@ public class CaseWhenEval extends EvalNode implements GsonObject {
@Override
public DataType getValueType() {
- return whens.get(0).getResultExpr().getValueType();
+ return whens.get(0).getResult().getValueType();
}
@Override
@@ -163,11 +163,19 @@ public class CaseWhenEval extends EvalNode implements GsonObject {
return result.eval(schema, tuple);
}
- public EvalNode getConditionExpr() {
+ public void setCondition(EvalNode condition) {
+ this.condition = condition;
+ }
+
+ public EvalNode getCondition() {
return this.condition;
}
- public EvalNode getResultExpr() {
+ public void setResult(EvalNode result) {
+ this.result = result;
+ }
+
+ public EvalNode getResult() {
return this.result;
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java
index a024b01..24a0409 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/CastEval.java
@@ -36,6 +36,10 @@ public class CastEval extends EvalNode {
this.target = target;
}
+ public void setOperand(EvalNode operand) {
+ this.operand = operand;
+ }
+
public EvalNode getOperand() {
return operand;
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
index 7dcc26a..38c7e1f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/EvalTreeUtil.java
@@ -57,11 +57,69 @@ public class EvalTreeUtil {
if (evalNode.equals(target)) {
EvalNode parent = stack.peek();
- if (parent.getLeftExpr().equals(evalNode)) {
- parent.setLeftExpr(tobeReplaced);
- }
- if (parent.getRightExpr().equals(evalNode)) {
- parent.setRightExpr(tobeReplaced);
+ if (parent instanceof BetweenPredicateEval) {
+ BetweenPredicateEval between = (BetweenPredicateEval) parent;
+ if (between.getPredicand().equals(evalNode)) {
+ between.setPredicand(tobeReplaced);
+ }
+ if (between.getBegin().equals(evalNode)) {
+ between.setBegin(tobeReplaced);
+ }
+ if (between.getEnd().equals(evalNode)) {
+ between.setEnd(tobeReplaced);
+ }
+
+ } else if (parent instanceof CaseWhenEval) {
+ CaseWhenEval caseWhen = (CaseWhenEval) parent;
+
+ // Here, we need to only consider only 'Else'
+ // because IfElseEval is handled in the below condition.
+ if (caseWhen.hasElse() && caseWhen.getElse().equals(evalNode)) {
+ caseWhen.setElseResult(tobeReplaced);
+ }
+ } else if (parent instanceof CaseWhenEval.IfThenEval) {
+ CaseWhenEval.IfThenEval ifThen = (CaseWhenEval.IfThenEval) parent;
+ if (ifThen.getCondition().equals(evalNode)) {
+ ifThen.setCondition(tobeReplaced);
+ }
+ if (ifThen.getResult().equals(evalNode)) {
+ ifThen.setResult(tobeReplaced);
+ }
+ } else if (parent instanceof CastEval) {
+ CastEval cast = (CastEval) parent;
+ if (cast.getOperand().equals(evalNode)) {
+ cast.setOperand(tobeReplaced);
+ }
+
+ } else if (parent instanceof FunctionEval) {
+ FunctionEval functionEval = (FunctionEval) parent;
+ EvalNode [] arguments = functionEval.getArgs();
+ for (int i = 0; i < arguments.length; i++) {
+ if (arguments[i].equals(evalNode)) {
+ arguments[i] = tobeReplaced;
+ }
+ }
+ functionEval.setArgs(arguments);
+
+ } else if (parent instanceof NotEval) {
+ NotEval not = (NotEval) parent;
+ if (not.getChild().equals(evalNode)) {
+ not.setChild(tobeReplaced);
+ }
+
+ } else if (parent instanceof SignedEval) {
+ SignedEval sign = (SignedEval) parent;
+ if (sign.getChild().equals(evalNode)) {
+ sign.setChild(tobeReplaced);
+ }
+
+ } else {
+ if (parent.getLeftExpr() != null && parent.getLeftExpr().equals(evalNode)) {
+ parent.setLeftExpr(tobeReplaced);
+ }
+ if (parent.getRightExpr() != null && parent.getRightExpr().equals(evalNode)) {
+ parent.setRightExpr(tobeReplaced);
+ }
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
index de982e5..6781c34 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/FunctionEval.java
@@ -64,7 +64,7 @@ public abstract class FunctionEval extends EvalNode implements Cloneable {
public boolean isDistinct() {
return funcDesc.getFuncType() == DISTINCT_AGGREGATION || funcDesc.getFuncType() == DISTINCT_UDA;
}
-
+
public EvalNode [] getArgs() {
return this.argEvals;
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java
index 1a16af4..20de725 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/NotEval.java
@@ -36,6 +36,10 @@ public class NotEval extends EvalNode implements Cloneable {
this.childEval = childEval;
}
+ public void setChild(EvalNode child) {
+ this.childEval = child;
+ }
+
public EvalNode getChild() {
return childEval;
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
index 3c2eeb4..33ea70f 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/SignedEval.java
@@ -39,6 +39,10 @@ public class SignedEval extends EvalNode implements Cloneable {
return negative;
}
+ public void setChild(EvalNode child) {
+ this.childEval = child;
+ }
+
public EvalNode getChild() {
return childEval;
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/java/org/apache/tajo/client/TestTajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/client/TestTajoDump.java b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoDump.java
new file mode 100644
index 0000000..e0d745e
--- /dev/null
+++ b/tajo-core/src/test/java/org/apache/tajo/client/TestTajoDump.java
@@ -0,0 +1,46 @@
+/**
+ * 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.client;
+
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.tajo.QueryTestCaseBase;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintWriter;
+
+public class TestTajoDump extends QueryTestCaseBase {
+
+ @Test
+ public void testDump1() throws Exception {
+ if (!testingCluster.isHCatalogStoreRunning()) {
+ executeString("CREATE TABLE \"" + getCurrentDatabase() +
+ "\".\"TableName1\" (\"Age\" int, \"FirstName\" TEXT, lastname TEXT)");
+
+ UserGroupInformation userInfo = UserGroupInformation.getCurrentUser();
+ ByteArrayOutputStream bos = new ByteArrayOutputStream();
+ PrintWriter printWriter = new PrintWriter(bos);
+ TajoDump.dump(client, userInfo, getCurrentDatabase(), false, false, false, printWriter);
+ printWriter.flush();
+ printWriter.close();
+ assertStrings(new String(bos.toByteArray()));
+ bos.close();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
index d11cad9..9cc2410 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestTablePartitions.java
@@ -152,6 +152,41 @@ public class TestTablePartitions extends QueryTestCaseBase {
res = executeFile("case3.sql");
assertResultSet(res, "case3.result");
res.close();
+
+ // select pow(key, 2) from testQueryCasesOnColumnPartitionedTable
+ res = executeFile("case4.sql");
+ assertResultSet(res, "case4.result");
+ res.close();
+
+ // select round(pow(key + 1, 2)) from testQueryCasesOnColumnPartitionedTable
+ res = executeFile("case5.sql");
+ assertResultSet(res, "case5.result");
+ res.close();
+
+ // select col1, key from testQueryCasesOnColumnPartitionedTable order by pow(key, 2) desc
+ res = executeFile("case6.sql");
+ assertResultSet(res, "case6.result");
+ res.close();
+
+ // select col1, key from testQueryCasesOnColumnPartitionedTable WHERE key BETWEEN 35 AND 48;
+ res = executeFile("case7.sql");
+ assertResultSet(res, "case7.result");
+ res.close();
+
+ // select col1, CASE key WHEN 36 THEN key WHEN 49 THEN key ELSE key END from testQueryCasesOnColumnPartitionedTable;
+ res = executeFile("case8.sql");
+ assertResultSet(res, "case8.result");
+ res.close();
+
+ // select col1, CAST(key AS INT) from testQueryCasesOnColumnPartitionedTable;
+ res = executeFile("case9.sql");
+ assertResultSet(res, "case9.result");
+ res.close();
+
+ // select col1, (!(key > 35)) from testQueryCasesOnColumnPartitionedTable;
+ res = executeFile("case10.sql");
+ assertResultSet(res, "case10.result");
+ res.close();
}
@Test
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/org/apache/tajo/client/TestTajoDump.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/org/apache/tajo/client/TestTajoDump.java b/tajo-core/src/test/resources/org/apache/tajo/client/TestTajoDump.java
deleted file mode 100644
index 7e72f9b..0000000
--- a/tajo-core/src/test/resources/org/apache/tajo/client/TestTajoDump.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tajo.client;
-
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.tajo.QueryTestCaseBase;
-import org.junit.Test;
-
-import java.io.ByteArrayOutputStream;
-import java.io.PrintWriter;
-
-public class TestTajoDump extends QueryTestCaseBase {
-
- @Test
- public void testDump1() throws Exception {
- executeString("CREATE TABLE \"" +getCurrentDatabase() +
- "\".\"TableName1\" (\"Age\" int, \"FirstName\" TEXT, lastname TEXT)");
-
- UserGroupInformation userInfo = UserGroupInformation.getCurrentUser();
- ByteArrayOutputStream bos = new ByteArrayOutputStream();
- PrintWriter printWriter = new PrintWriter(bos);
- TajoDump.dump(client, userInfo, getCurrentDatabase(), false, false, printWriter);
- printWriter.flush();
- printWriter.close();
- assertStrings(new String(bos.toByteArray()));
- bos.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/queries/TestTablePartitions/case10.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case10.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case10.sql
new file mode 100644
index 0000000..715d8e8
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case10.sql
@@ -0,0 +1 @@
+select col1, key, -key as res from testQueryCasesOnColumnPartitionedTable;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/queries/TestTablePartitions/case8.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case8.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case8.sql
new file mode 100644
index 0000000..fe57eb8
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case8.sql
@@ -0,0 +1 @@
+select col1, CASE key WHEN 36 THEN key WHEN 49 THEN key ELSE key END from testQueryCasesOnColumnPartitionedTable;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/queries/TestTablePartitions/case9.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestTablePartitions/case9.sql b/tajo-core/src/test/resources/queries/TestTablePartitions/case9.sql
new file mode 100644
index 0000000..595e533
--- /dev/null
+++ b/tajo-core/src/test/resources/queries/TestTablePartitions/case9.sql
@@ -0,0 +1 @@
+select col1, CAST(key AS INT) from testQueryCasesOnColumnPartitionedTable;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/results/TestTablePartitions/case10.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case10.result b/tajo-core/src/test/resources/results/TestTablePartitions/case10.result
new file mode 100644
index 0000000..a6be56d
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case10.result
@@ -0,0 +1,7 @@
+col1,key,res
+-------------------------------
+1,17.0,-17.0
+1,36.0,-36.0
+2,38.0,-38.0
+3,45.0,-45.0
+3,49.0,-49.0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/results/TestTablePartitions/case7.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case7.result b/tajo-core/src/test/resources/results/TestTablePartitions/case7.result
new file mode 100644
index 0000000..8426f48
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case7.result
@@ -0,0 +1,5 @@
+col1,key
+-------------------------------
+1,36.0
+2,38.0
+3,45.0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/results/TestTablePartitions/case8.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case8.result b/tajo-core/src/test/resources/results/TestTablePartitions/case8.result
new file mode 100644
index 0000000..cb51cd6
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case8.result
@@ -0,0 +1,7 @@
+col1,?casewhen
+-------------------------------
+1,17.0
+1,36.0
+2,38.0
+3,45.0
+3,49.0
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/results/TestTablePartitions/case9.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTablePartitions/case9.result b/tajo-core/src/test/resources/results/TestTablePartitions/case9.result
new file mode 100644
index 0000000..ebfaad2
--- /dev/null
+++ b/tajo-core/src/test/resources/results/TestTablePartitions/case9.result
@@ -0,0 +1,7 @@
+col1,?cast
+-------------------------------
+1,17
+1,36
+2,38
+3,45
+3,49
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/results/TestTajoDump/testDump1.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestTajoDump/testDump1.result b/tajo-core/src/test/resources/results/TestTajoDump/testDump1.result
index bdf31fa..fa68435 100644
--- a/tajo-core/src/test/resources/results/TestTajoDump/testDump1.result
+++ b/tajo-core/src/test/resources/results/TestTajoDump/testDump1.result
@@ -1,9 +1,6 @@
--
-- Tajo database dump
--
--- Dump user: hyunsik
--- Dump date: 04/17/2014 13:33:45
---
--
@@ -11,7 +8,9 @@
--
CREATE DATABASE IF NOT EXISTS "TestTajoDump";
+
--
--- Name: TestTajoDump.TableName1; Type: TABLE; Storage: CSV
+-- Name: "TestTajoDump"."TableName1"; Type: TABLE; Storage: CSV
--
-CREATE TABLE "TestTajoDump"."TableName1" ("Age" INT4, "FirstName" TEXT, lastname TEXT) USING CSV WITH ('csvfile.delimiter'='|');
\ No newline at end of file
+CREATE TABLE "TestTajoDump"."TableName1" ("Age" INT4, "FirstName" TEXT, lastname TEXT) USING CSV WITH ('csvfile.delimiter'='|');
+
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result b/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result
index bd59e2d..a6e2d05 100644
--- a/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result
+++ b/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLForBaseTable.result
@@ -1,5 +1,4 @@
--
-- Name: db1.table2; Type: TABLE; Storage: CSV
--- Path: /table1
--
CREATE TABLE db1.table2 (name BLOB, addr TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|');
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/e56a7a41/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result b/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result
index da391cb..eeb9b59 100644
--- a/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result
+++ b/tajo-core/src/test/resources/results/testDDLBuilder/testBuildDDLQuotedTableName2.result
@@ -1,5 +1,4 @@
--
-- Name: db1."TABLE1"; Type: TABLE; Storage: CSV
--- Path: /table1
--
CREATE TABLE db1."TABLE1" (name BLOB, addr TEXT, "FirstName" TEXT, "LastName" TEXT, "with" TEXT) USING CSV WITH ('compression.codec'='org.apache.hadoop.io.compress.GzipCodec', 'csvfile.delimiter'='|') PARTITION BY COLUMN("BirthYear" INT4);
\ No newline at end of file
[5/9] git commit: TAJO-748: Shuffle output numbers of join may be
inconsistent. (jaehwa)
Posted by hy...@apache.org.
TAJO-748: Shuffle output numbers of join may be inconsistent. (jaehwa)
Project: http://git-wip-us.apache.org/repos/asf/tajo/repo
Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/2b27f7de
Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/2b27f7de
Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/2b27f7de
Branch: refs/heads/window_function
Commit: 2b27f7de70904552d38801f57aa12396a9df75ac
Parents: f1f36ec
Author: blrunner <jh...@gruter.com>
Authored: Mon Apr 21 16:11:13 2014 +0900
Committer: blrunner <jh...@gruter.com>
Committed: Mon Apr 21 16:11:13 2014 +0900
----------------------------------------------------------------------
CHANGES.txt | 2 +
.../java/org/apache/tajo/conf/TajoConf.java | 2 +-
.../engine/planner/global/GlobalPlanner.java | 49 +++++++++++++++++++-
.../tajo/master/querymaster/SubQuery.java | 18 +++++++
.../tajo/engine/query/TestJoinBroadcast.java | 14 +++---
.../tajo/master/TestExecutionBlockCursor.java | 2 +-
.../querymaster/TestQueryUnitStatusUpdate.java | 8 ++--
.../resources/queries/TestNetTypes/testJoin.sql | 2 +-
.../testBroadcastSubquery2.result | 2 +-
9 files changed, 82 insertions(+), 17 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d6ed95a..0fcc83d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -334,6 +334,8 @@ Release 0.8.0 - unreleased
BUG FIXES
+ TAJO-748: Shuffle output numbers of join may be inconsistent. (jaehwa)
+
TAJO-777: Partition column in function parameter occurs NPE.
(Hyoungjun Kim via hyunsik)
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
----------------------------------------------------------------------
diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
index 3c81ed5..5b3d4b3 100644
--- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
+++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java
@@ -161,7 +161,7 @@ public class TajoConf extends Configuration {
//////////////////////////////////////////
// Distributed Query Execution Parameters
//////////////////////////////////////////
- DIST_QUERY_BROADCAST_JOIN_AUTO("tajo.dist-query.join.broadcast.auto", true),
+ DIST_QUERY_BROADCAST_JOIN_AUTO("tajo.dist-query.join.auto-broadcast", true),
DIST_QUERY_BROADCAST_JOIN_THRESHOLD("tajo.dist-query.join.broadcast.threshold-bytes", (long)5 * 1048576),
DIST_QUERY_JOIN_TASK_VOLUME("tajo.dist-query.join.task-volume-mb", 128),
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
index bf2bf7d..edc08fc 100644
--- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
+++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/GlobalPlanner.java
@@ -210,6 +210,10 @@ public class GlobalPlanner {
throw new PlanningException("Invalid State");
}
+ private static boolean checkIfCanBeOneOfBroadcastJoin(LogicalNode node) {
+ return node.getType() == NodeType.SCAN || node.getType() == NodeType.PARTITIONS_SCAN;
+ }
+
private ExecutionBlock buildJoinPlan(GlobalPlanContext context, JoinNode joinNode,
ExecutionBlock leftBlock, ExecutionBlock rightBlock)
throws PlanningException {
@@ -218,6 +222,7 @@ public class GlobalPlanner {
boolean autoBroadcast = conf.getBoolVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_AUTO);
+ // to check when the tajo.dist-query.join.broadcast.auto property is true
if (autoBroadcast && joinNode.isCandidateBroadcast()) {
long broadcastThreshold = conf.getLongVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD);
List<LogicalNode> broadtargetTables = new ArrayList<LogicalNode>();
@@ -234,7 +239,6 @@ public class GlobalPlanner {
}
}
- //large table must be one
if (numLargeTables <= 1 && !broadtargetTables.isEmpty()) {
currentBlock = masterPlan.newExecutionBlock();
currentBlock.setPlan(joinNode);
@@ -250,6 +254,49 @@ public class GlobalPlanner {
}
}
+ LogicalNode leftNode = joinNode.getLeftChild();
+ LogicalNode rightNode = joinNode.getRightChild();
+
+ boolean leftBroadcasted = false;
+ boolean rightBroadcasted = false;
+
+ // Although broadcast join property is false, we need to handle boradcast join.
+ // It must, Shuffle output numbers of join will be consistent.
+ if (checkIfCanBeOneOfBroadcastJoin(leftNode) && checkIfCanBeOneOfBroadcastJoin(rightNode)) {
+ ScanNode leftScan = (ScanNode) leftNode;
+ ScanNode rightScan = (ScanNode) rightNode;
+
+ TableDesc leftDesc = leftScan.getTableDesc();
+ TableDesc rightDesc = rightScan.getTableDesc();
+ long broadcastThreshold = conf.getLongVar(TajoConf.ConfVars.DIST_QUERY_BROADCAST_JOIN_THRESHOLD);
+
+ if (leftDesc.getStats().getNumBytes() < broadcastThreshold) {
+ leftBroadcasted = true;
+ }
+ if (rightDesc.getStats().getNumBytes() < broadcastThreshold) {
+ rightBroadcasted = true;
+ }
+
+ if (leftBroadcasted || rightBroadcasted) {
+ currentBlock = masterPlan.newExecutionBlock();
+ currentBlock.setPlan(joinNode);
+ if (leftBroadcasted) {
+ currentBlock.addBroadcastTable(leftScan.getCanonicalName());
+ LOG.info("The left table " + rightScan.getCanonicalName() + " ("
+ + rightScan.getTableDesc().getStats().getNumBytes() + ") is marked a broadcasted table");
+ }
+ if (rightBroadcasted) {
+ currentBlock.addBroadcastTable(rightScan.getCanonicalName());
+ LOG.info("The right table " + rightScan.getCanonicalName() + " ("
+ + rightScan.getTableDesc().getStats().getNumBytes() + ") is marked a broadcasted table");
+ }
+
+ context.execBlockMap.remove(leftScan.getPID());
+ context.execBlockMap.remove(rightScan.getPID());
+ return currentBlock;
+ }
+ }
+
// symmetric repartition join
currentBlock = masterPlan.newExecutionBlock();
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
index 63b50ac..8929e8d 100644
--- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
+++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/SubQuery.java
@@ -742,6 +742,24 @@ public class SubQuery implements EventHandler<SubQueryEvent> {
taskNum = Math.min(taskNum, slots);
LOG.info(subQuery.getId() + ", The determined number of join partitions is " + taskNum);
+ // The shuffle output numbers of join may be inconsistent by execution block order.
+ // Thus, we need to compare the number with DataChannel output numbers.
+ // If the number is right, the number and DataChannel output numbers will be consistent.
+ int outerShuffleOutptNum = 0, innerShuffleOutputNum = 0;
+ for (DataChannel eachChannel : masterPlan.getOutgoingChannels(outer.getId())) {
+ outerShuffleOutptNum = Math.max(outerShuffleOutptNum, eachChannel.getShuffleOutputNum());
+ }
+
+ for (DataChannel eachChannel : masterPlan.getOutgoingChannels(inner.getId())) {
+ innerShuffleOutputNum = Math.max(innerShuffleOutputNum, eachChannel.getShuffleOutputNum());
+ }
+
+ if (outerShuffleOutptNum != innerShuffleOutputNum
+ && taskNum != outerShuffleOutptNum
+ && taskNum != innerShuffleOutputNum) {
+ taskNum = Math.max(outerShuffleOutptNum, innerShuffleOutputNum);
+ }
+
return taskNum;
// Is this subquery the first step of group-by?
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
index 89519ef..f5f98a5 100644
--- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
+++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinBroadcast.java
@@ -365,13 +365,11 @@ public class TestJoinBroadcast extends QueryTestCaseBase {
cleanupQuery(res);
}
- // It doesn't run as expected because of TAJO-747 bug.
- // Thus, we need to block this method until resolving this bug.
-// @Test
-// public final void testBroadcastSubquery2() throws Exception {
-// ResultSet res = executeQuery();
-// assertResultSet(res);
-// cleanupQuery(res);
-// }
+ @Test
+ public final void testBroadcastSubquery2() throws Exception {
+ ResultSet res = executeQuery();
+ assertResultSet(res);
+ cleanupQuery(res);
+ }
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
index ab31c8d..f4fa74a 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/TestExecutionBlockCursor.java
@@ -118,6 +118,6 @@ public class TestExecutionBlockCursor {
count++;
}
- assertEquals(10, count);
+ assertEquals(6, count);
}
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
index 07b4ac5..fa89dc3 100644
--- a/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
+++ b/tajo-core/src/test/java/org/apache/tajo/master/querymaster/TestQueryUnitStatusUpdate.java
@@ -86,11 +86,11 @@ public class TestQueryUnitStatusUpdate extends QueryTestCaseBase {
res = executeQuery();
- long[] expectedNumRows = new long[]{2, 2, 5, 5, 7, 2, 2, 2};
- long[] expectedNumBytes = new long[]{18, 34, 45, 75, 109, 34, 34, 18};
- long[] expectedReadBytes = new long[]{18, 0, 45, 0, 109, 0, 34, 0};
+ long[] expectedNumRows = new long[]{7, 2, 2, 2, 7, 2, 2, 2};
+ long[] expectedNumBytes = new long[]{63, 34, 34, 18, 109, 34, 34, 18};
+ long[] expectedReadBytes = new long[]{63, 0, 34, 0, 109, 0, 34, 0};
- assertStatus(4, expectedNumRows, expectedNumBytes, expectedReadBytes);
+ assertStatus(2, expectedNumRows, expectedNumBytes, expectedReadBytes);
} finally {
cleanupQuery(res);
}
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-core/src/test/resources/queries/TestNetTypes/testJoin.sql
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/queries/TestNetTypes/testJoin.sql b/tajo-core/src/test/resources/queries/TestNetTypes/testJoin.sql
index ec4f8e6..22c97d5 100644
--- a/tajo-core/src/test/resources/queries/TestNetTypes/testJoin.sql
+++ b/tajo-core/src/test/resources/queries/TestNetTypes/testJoin.sql
@@ -1 +1 @@
-select t1.*,t2.* from table1 as t1, table2 as t2 where t1.addr = t2.addr order by t2.name;
\ No newline at end of file
+select t1.*,t2.* from table1 as t1, table2 as t2 where t1.addr = t2.addr order by t1.id, t1.name,t2. name;
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tajo/blob/2b27f7de/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastSubquery2.result
----------------------------------------------------------------------
diff --git a/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastSubquery2.result b/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastSubquery2.result
index 14c2211..9368976 100644
--- a/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastSubquery2.result
+++ b/tajo-core/src/test/resources/results/TestJoinBroadcast/testBroadcastSubquery2.result
@@ -1,3 +1,3 @@
?sum
-------------------------------
-360.0
\ No newline at end of file
+190.0
\ No newline at end of file