Return-Path: X-Original-To: apmail-tajo-commits-archive@minotaur.apache.org Delivered-To: apmail-tajo-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7A24D11206 for ; Thu, 5 Jun 2014 00:45:42 +0000 (UTC) Received: (qmail 90508 invoked by uid 500); 5 Jun 2014 00:45:42 -0000 Delivered-To: apmail-tajo-commits-archive@tajo.apache.org Received: (qmail 90474 invoked by uid 500); 5 Jun 2014 00:45:42 -0000 Mailing-List: contact commits-help@tajo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@tajo.apache.org Delivered-To: mailing list commits@tajo.apache.org Received: (qmail 90467 invoked by uid 99); 5 Jun 2014 00:45:42 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 05 Jun 2014 00:45:42 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 2314A94A245; Thu, 5 Jun 2014 00:45:42 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: hyunsik@apache.org To: commits@tajo.apache.org Message-Id: <025d342112a84581b1bb9720c662fdaa@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: git commit: TAJO-762: Implements current date/time function (Hyoungjun Kim via hyunsik) Date: Thu, 5 Jun 2014 00:45:42 +0000 (UTC) Repository: tajo Updated Branches: refs/heads/master 64106a322 -> 26dc6e926 TAJO-762: Implements current date/time function (Hyoungjun Kim via hyunsik) Closes #16 Project: http://git-wip-us.apache.org/repos/asf/tajo/repo Commit: http://git-wip-us.apache.org/repos/asf/tajo/commit/26dc6e92 Tree: http://git-wip-us.apache.org/repos/asf/tajo/tree/26dc6e92 Diff: http://git-wip-us.apache.org/repos/asf/tajo/diff/26dc6e92 Branch: refs/heads/master Commit: 26dc6e926db3934ea7309cc4e93d24a5881de97d Parents: 64106a3 Author: Hyunsik Choi Authored: Wed Jun 4 17:43:56 2014 -0700 Committer: Hyunsik Choi Committed: Wed Jun 4 17:45:22 2014 -0700 ---------------------------------------------------------------------- CHANGES | 2 + .../org/apache/tajo/catalog/CatalogUtil.java | 2 +- .../java/org/apache/tajo/client/TajoClient.java | 8 ++- .../org/apache/tajo/engine/parser/SQLLexer.g4 | 4 ++ .../org/apache/tajo/engine/parser/SQLParser.g4 | 51 ++++++++++++- .../apache/tajo/engine/eval/AlgebraicUtil.java | 8 ++- .../tajo/engine/function/builtin/Today.java | 46 ------------ .../engine/function/datetime/CurrentDate.java | 57 +++++++++++++++ .../engine/function/datetime/CurrentTime.java | 56 +++++++++++++++ .../engine/function/datetime/NowTimestamp.java | 51 +++++++++++++ .../apache/tajo/engine/parser/SQLAnalyzer.java | 76 +++++++++++++++++--- .../tajo/engine/planner/ExprAnnotator.java | 2 +- .../tajo/master/querymaster/QueryMaster.java | 7 +- .../querymaster/QueryMasterManagerService.java | 7 +- .../master/querymaster/QueryMasterTask.java | 12 +++- .../master/rm/TajoWorkerResourceManager.java | 3 +- .../tajo/engine/eval/TestIntervalType.java | 9 ++- .../engine/function/TestBuiltinFunctions.java | 18 +++-- .../engine/function/TestDateTimeFunctions.java | 52 ++++++++++++++ .../tajo/engine/query/TestCreateTable.java | 29 ++++---- .../tajo/engine/query/TestInsertQuery.java | 9 ++- .../tajo/engine/query/TestJoinBroadcast.java | 3 + .../apache/tajo/engine/query/TestJoinQuery.java | 8 +++ 23 files changed, 420 insertions(+), 100 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/CHANGES ---------------------------------------------------------------------- diff --git a/CHANGES b/CHANGES index d2436e9..7316e4d 100644 --- a/CHANGES +++ b/CHANGES @@ -15,6 +15,8 @@ Release 0.9.0 - unreleased TAJO-791: Implements ADD_DAYS() function. (Hyoungjun Kim via hyunsik) + TAJO-762: Implements current date/time function (Hyoungjun Kim via hyunsik) + IMPROVEMENT TAJO-854: Supports INSERT INTO with UNION. (Hyoungjun Kim via jihoon) http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java ---------------------------------------------------------------------- diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java index 85ea516..331bdae 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java @@ -707,7 +707,7 @@ public class CatalogUtil { static final String [] RESERVED_KEYWORDS = { "AS", "ALL", "AND", "ANY", "ASYMMETRIC", "ASC", "BOTH", - "CASE", "CAST", "CREATE", "CROSS", + "CASE", "CAST", "CREATE", "CROSS", "CURRENT_DATE", "CURRENT_TIME", "CURRENT_TIMESTAMP", "DESC", "DISTINCT", "END", "ELSE", "EXCEPT", "FALSE", "FULL", "FROM", http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java ---------------------------------------------------------------------- diff --git a/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java b/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java index 2f9e138..333c8d6 100644 --- a/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java +++ b/tajo-client/src/main/java/org/apache/tajo/client/TajoClient.java @@ -34,6 +34,7 @@ import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.cli.InvalidClientSessionException; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.ipc.ClientProtos; import org.apache.tajo.ipc.ClientProtos.*; import org.apache.tajo.ipc.QueryMasterClientProtocol; import org.apache.tajo.ipc.QueryMasterClientProtocol.QueryMasterClientProtocolService; @@ -346,6 +347,9 @@ public class TajoClient implements Closeable { throws ServiceException, IOException { SubmitQueryResponse response = executeQuery(sql); + if (response.getResultCode() == ClientProtos.ResultCode.ERROR) { + throw new ServiceException(response.getErrorTrace()); + } QueryId queryId = new QueryId(response.getQueryId()); if (response.getIsForwarded()) { if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) { @@ -369,7 +373,9 @@ public class TajoClient implements Closeable { public ResultSet executeJsonQueryAndGetResult(final String json) throws ServiceException, IOException { SubmitQueryResponse response = executeQueryWithJson(json); - + if (response.getResultCode() == ClientProtos.ResultCode.ERROR) { + throw new ServiceException(response.getErrorTrace()); + } QueryId queryId = new QueryId(response.getQueryId()); if (response.getIsForwarded()) { if (queryId.equals(QueryIdFactory.NULL_QUERY_ID)) { http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 index 05312ce..e26f921 100644 --- a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 +++ b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLLexer.g4 @@ -128,6 +128,10 @@ CASE : C A S E; CAST : C A S T; CREATE : C R E A T E; CROSS : C R O S S; +CURRENT_DATE: C U R R E N T UNDERLINE D A T E; +CURRENT_TIME: C U R R E N T UNDERLINE T I M E; +CURRENT_TIMESTAMP: C U R R E N T UNDERLINE T I M E S T A M P; + DESC : D E S C; DISTINCT : D I S T I N C T; http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 index 9570457..9a8f38e 100644 --- a/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 +++ b/tajo-core/src/main/antlr4/org/apache/tajo/engine/parser/SQLParser.g4 @@ -653,6 +653,7 @@ value_expression common_value_expression : numeric_value_expression | string_value_expression + | datetime_value_expression | NULL ; @@ -714,8 +715,7 @@ time_zone_field ; extract_source - : column_reference - | datetime_literal + : datetime_value_expression ; /* @@ -766,6 +766,53 @@ trim_specification /* =============================================================================== + 6.30 +=============================================================================== +*/ +datetime_value_expression + : datetime_term + ; +datetime_term + : datetime_factor + ; + +datetime_factor + : datetime_primary + ; + +datetime_primary + : value_expression_primary + | datetime_value_function + ; + +/* +=============================================================================== + 6.31 +=============================================================================== +*/ + +datetime_value_function + : current_date_value_function + | current_time_value_function + | current_timestamp_value_function + ; + +current_date_value_function + : CURRENT_DATE + | CURRENT_DATE LEFT_PAREN RIGHT_PAREN + ; + +current_time_value_function + : CURRENT_TIME + | CURRENT_TIME LEFT_PAREN RIGHT_PAREN + ; + +current_timestamp_value_function + : CURRENT_TIMESTAMP + ; + +/* +=============================================================================== 6.34 =============================================================================== */ http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java index 1cb37db..c04e74f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/eval/AlgebraicUtil.java @@ -171,8 +171,12 @@ public class AlgebraicUtil { public EvalNode visitFuncCall(Object context, GeneralFunctionEval evalNode, Stack stack) { boolean constant = true; - for (EvalNode arg : evalNode.getArgs()) { - constant &= (arg.getType() == EvalType.CONST); + if ("sleep".equals(evalNode.funcDesc.getSignature())) { + constant = false; + } else { + for (EvalNode arg : evalNode.getArgs()) { + constant &= (arg.getType() == EvalType.CONST); + } } if (constant) { http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java deleted file mode 100644 index 157e545..0000000 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/builtin/Today.java +++ /dev/null @@ -1,46 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.engine.function.builtin; - -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.engine.function.GeneralFunction; -import org.apache.tajo.engine.function.annotation.Description; -import org.apache.tajo.engine.function.annotation.ParamTypes; -import org.apache.tajo.storage.Tuple; - -@Description( - functionName = "today", - description = "get current time millis", - example = "> SELECT today();", - returnType = TajoDataTypes.Type.INT8, - paramTypes = {@ParamTypes(paramTypes = {})} -) -public class Today extends GeneralFunction { - - public Today() { - super(NoArgs); - } - - @Override - public Datum eval(Tuple params) { - return DatumFactory.createInt8(System.currentTimeMillis()); - } -} http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java new file mode 100644 index 0000000..2b3fcdf --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentDate.java @@ -0,0 +1,57 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.datetime; + +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.DateDatum; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.DatumFactory; +import org.apache.tajo.engine.function.GeneralFunction; +import org.apache.tajo.engine.function.annotation.Description; +import org.apache.tajo.engine.function.annotation.ParamTypes; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.util.datetime.DateTimeUtil; +import org.apache.tajo.util.datetime.TimeMeta; + +@Description( + functionName = "current_date", + description = "Get current date. Result is DATE type.", + example = "> SELECT current_date();\n2014-04-18", + returnType = TajoDataTypes.Type.DATE, + paramTypes = {@ParamTypes(paramTypes = {})} +) +public class CurrentDate extends GeneralFunction { + DateDatum datum; + + public CurrentDate() { + super(NoArgs); + } + + @Override + public Datum eval(Tuple params) { + if (datum == null) { + long julianTimestamp = DateTimeUtil.javaTimeToJulianTime(System.currentTimeMillis()); + TimeMeta tm = new TimeMeta(); + DateTimeUtil.toJulianTimeMeta(julianTimestamp, tm); + DateTimeUtil.toUserTimezone(tm); + datum = DatumFactory.createDate(tm.years, tm.monthOfYear, tm.dayOfMonth); + } + return datum; + } +} http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentTime.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentTime.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentTime.java new file mode 100644 index 0000000..1efdfa9 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/CurrentTime.java @@ -0,0 +1,56 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.datetime; + +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.DatumFactory; +import org.apache.tajo.datum.TimeDatum; +import org.apache.tajo.engine.function.GeneralFunction; +import org.apache.tajo.engine.function.annotation.Description; +import org.apache.tajo.engine.function.annotation.ParamTypes; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.util.datetime.DateTimeUtil; +import org.apache.tajo.util.datetime.TimeMeta; + +@Description( + functionName = "current_time", + description = "Get current time. Result is TIME type.", + example = "> SELECT current_time();\n12:30:40", + returnType = TajoDataTypes.Type.TIME, + paramTypes = {@ParamTypes(paramTypes = {})} +) +public class CurrentTime extends GeneralFunction { + TimeDatum datum; + + public CurrentTime() { + super(NoArgs); + } + + @Override + public Datum eval(Tuple params) { + if (datum == null) { + long julianTimestamp = DateTimeUtil.javaTimeToJulianTime(System.currentTimeMillis()); + TimeMeta tm = new TimeMeta(); + DateTimeUtil.toJulianTimeMeta(julianTimestamp, tm); + datum = DatumFactory.createTime(DateTimeUtil.toTime(tm)); + } + return datum; + } +} http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/NowTimestamp.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/NowTimestamp.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/NowTimestamp.java new file mode 100644 index 0000000..adc093c --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/datetime/NowTimestamp.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.datetime; + +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.DatumFactory; +import org.apache.tajo.datum.TimestampDatum; +import org.apache.tajo.engine.function.GeneralFunction; +import org.apache.tajo.engine.function.annotation.Description; +import org.apache.tajo.engine.function.annotation.ParamTypes; +import org.apache.tajo.storage.Tuple; + +@Description( + functionName = "now", + description = "Get current time. Result is TIMESTAMP type.", + example = "> SELECT now();\n2014-04-18 22:54:29.280", + returnType = TajoDataTypes.Type.TIMESTAMP, + paramTypes = {@ParamTypes(paramTypes = {})} +) +public class NowTimestamp extends GeneralFunction { + TimestampDatum datum; + + public NowTimestamp() { + super(NoArgs); + } + + @Override + public Datum eval(Tuple params) { + if (datum == null) { + datum = DatumFactory.createTimestmpDatumWithJavaMillis(System.currentTimeMillis()); + } + return datum; + } +} http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java index fe65d47..f0b5aa0 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java @@ -947,14 +947,15 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { @Override public Expr visitExtract_expression(Extract_expressionContext ctx) { Expr extractTarget = new LiteralValue(ctx.extract_field_string.getText(), LiteralType.String); - Expr extractSource; - if (checkIfExist(ctx.extract_source().column_reference())) { - extractSource = visitColumn_reference(ctx.extract_source().column_reference()); - } else if (checkIfExist(ctx.extract_source().datetime_literal())) { - extractSource = visitDatetime_literal(ctx.extract_source().datetime_literal()); - } else { - return null; - } + Expr extractSource = visitDatetime_value_expression(ctx.extract_source().datetime_value_expression()); +// if (checkIfExist(ctx.extract_source().column_reference())) { +// extractSource = visitColumn_reference(ctx.extract_source().column_reference()); +// } else if (checkIfExist(ctx.extract_source().datetime_literal())) { +// extractSource = visitDatetime_literal(ctx.extract_source().datetime_literal()); +// } else { +// return null; +// } + String functionName = "date_part"; Expr[] params = new Expr[]{extractTarget, extractSource}; @@ -1416,11 +1417,68 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return new TimestampLiteral(parseDate(datePart), parseTime(timePart)); } - @Override public Expr visitInterval_literal(@NotNull SQLParser.Interval_literalContext ctx) { + @Override + public Expr visitInterval_literal(@NotNull SQLParser.Interval_literalContext ctx) { String intervalStr = stripQuote(ctx.interval_string.getText()); return new IntervalLiteral(intervalStr); } + @Override + public Expr visitDatetime_value_expression(@NotNull SQLParser.Datetime_value_expressionContext ctx) { + return visitDatetime_term(ctx.datetime_term()); + } + + @Override + public Expr visitDatetime_term(@NotNull SQLParser.Datetime_termContext ctx) { + return visitDatetime_factor(ctx.datetime_factor()); + } + + @Override + public Expr visitDatetime_factor(@NotNull SQLParser.Datetime_factorContext ctx) { + return visitDatetime_primary(ctx.datetime_primary()); + } + + @Override + public Expr visitDatetime_primary(@NotNull SQLParser.Datetime_primaryContext ctx) { + if (checkIfExist(ctx.value_expression_primary())) { + return visitValue_expression_primary(ctx.value_expression_primary()); + } else { + return visitDatetime_value_function(ctx.datetime_value_function()); + } + } + + @Override + public Expr visitDatetime_value_function(@NotNull SQLParser.Datetime_value_functionContext ctx) { + if (checkIfExist(ctx.current_date_value_function())) { + return visitCurrent_date_value_function(ctx.current_date_value_function()); + } else if (checkIfExist(ctx.current_time_value_function())) { + return visitCurrent_time_value_function(ctx.current_time_value_function()); + } else { + return visitCurrent_timestamp_value_function(ctx.current_timestamp_value_function()); + } + } + + @Override + public Expr visitCurrent_date_value_function(@NotNull SQLParser.Current_date_value_functionContext ctx) { + String functionName = "current_date"; + Expr[] params = new Expr[]{}; + return new FunctionExpr(functionName, params); + } + + @Override + public Expr visitCurrent_time_value_function(@NotNull SQLParser.Current_time_value_functionContext ctx) { + String functionName = "current_time"; + Expr[] params = new Expr[]{}; + return new FunctionExpr(functionName, params); + } + + @Override + public Expr visitCurrent_timestamp_value_function(@NotNull SQLParser.Current_timestamp_value_functionContext ctx) { + String functionName = "now"; + Expr[] params = new Expr[]{}; + return new FunctionExpr(functionName, params); + } + private DateValue parseDate(String datePart) { // e.g., 1980-04-01 String[] parts = datePart.split("-"); http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java index ba80c25..e799e30 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/ExprAnnotator.java @@ -68,7 +68,7 @@ public class ExprAnnotator extends BaseAlgebraVisitor(), expr); + return AlgebraicUtil.eliminateConstantExprs(visit(context, new Stack(), expr)); } public static void assertEval(boolean condition, String message) throws PlanningException { http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java index a8c6014..f173c24 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMaster.java @@ -42,7 +42,6 @@ import org.apache.tajo.rpc.RpcConnectionPool; import org.apache.tajo.rpc.protocolrecords.PrimitiveProtos; import org.apache.tajo.storage.AbstractStorageManager; import org.apache.tajo.storage.StorageManagerFactory; -import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.NetUtils; import org.apache.tajo.worker.TajoWorker; @@ -337,10 +336,10 @@ public class QueryMaster extends CompositeService implements EventHandler { try { queryMasterTask.stop(); - if (!systemConf.get(CommonTestingUtil.TAJO_TEST, "FALSE").equalsIgnoreCase("TRUE") - && !workerContext.isYarnContainerMode()) { + //if (!systemConf.get(CommonTestingUtil.TAJO_TEST, "FALSE").equalsIgnoreCase("TRUE") + // && !workerContext.isYarnContainerMode()) { cleanup(queryId); // TODO We will support yarn mode - } + //} } catch (Exception e) { LOG.error(e.getMessage(), e); } http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java index 589a656..826052d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterManagerService.java @@ -217,7 +217,12 @@ public class QueryMasterManagerService extends CompositeService RpcCallback done) { QueryId queryId = new QueryId(request); QueryMasterTask queryMasterTask = queryMaster.getQueryMasterTask(queryId); - queryMasterTask.getQuery().handle(new QueryEvent(queryId, QueryEventType.KILL)); + if (queryMasterTask != null) { + Query query = queryMasterTask.getQuery(); + if (query != null) { + query.handle(new QueryEvent(queryId, QueryEventType.KILL)); + } + } } @Override http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java index f812715..ecf2202 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/querymaster/QueryMasterTask.java @@ -210,7 +210,9 @@ public class QueryMasterTask extends CompositeService { super.stop(); //TODO change report to tajo master - queryMetrics.report(new MetricsConsoleReporter()); + if (queryMetrics != null) { + queryMetrics.report(new MetricsConsoleReporter()); + } LOG.info("Stopped QueryMasterTask:" + queryId); } @@ -392,8 +394,12 @@ public class QueryMasterTask extends CompositeService { } } catch (IOException ioe) { if (stagingDir != null && defaultFS.exists(stagingDir)) { - defaultFS.delete(stagingDir, true); - LOG.info("The staging directory '" + stagingDir + "' is deleted"); + try { + defaultFS.delete(stagingDir, true); + LOG.info("The staging directory '" + stagingDir + "' is deleted"); + } catch (Exception e) { + LOG.warn(e.getMessage()); + } } throw ioe; http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java index bb9f07d..18f2d24 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/rm/TajoWorkerResourceManager.java @@ -530,7 +530,6 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke @Override public void stopQueryMaster(QueryId queryId) { - WorkerResource resource = null; if(!rmContext.getQueryMasterContainer().containsKey(queryId)) { LOG.warn("No QueryMaster resource info for " + queryId); return; @@ -538,7 +537,7 @@ public class TajoWorkerResourceManager extends CompositeService implements Worke ContainerIdProto containerId = rmContext.getQueryMasterContainer().remove(queryId); releaseWorkerResource(containerId); rmContext.getStoppedQueryIds().add(queryId); - LOG.info(String.format("Released QueryMaster (%s) resource:" + resource, queryId.toString())); + LOG.info(String.format("Released QueryMaster (%s) resource." , queryId.toString())); } } http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java index 6450028..c054fd1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestIntervalType.java @@ -103,7 +103,14 @@ public class TestIntervalType extends ExprTestBase { testSimpleEval("select interval '1 month' * 2.3", new String[]{"2 months 9 days"}); testSimpleEval("select interval '3 year 5 month 1 hour' / 1.5", new String[]{"2 years 3 months 10 days 00:40:00"}); - testSimpleEval("select date '2001-09-28' - time '03:00'", new String[]{"2001-09-27 21:00:00" + getUserTimeZoneDisplay()}); + testSimpleEval("select date '2001-09-28' - time '03:00'", + new String[]{"2001-09-27 21:00:00" + getUserTimeZoneDisplay()}); + + testSimpleEval("select date '2014-03-20' + interval '1 day'", + new String[]{"2014-03-21 00:00:00" + getUserTimeZoneDisplay()}); + + testSimpleEval("select date '2014-03-20' - interval '1 day'", + new String[]{"2014-03-19 00:00:00" + getUserTimeZoneDisplay()}); } @Test http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java index 65612f7..4fefe07 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestBuiltinFunctions.java @@ -26,8 +26,6 @@ import org.junit.experimental.categories.Category; import java.sql.ResultSet; -import static org.junit.Assert.assertTrue; - @Category(IntegrationTest.class) public class TestBuiltinFunctions extends QueryTestCaseBase { @@ -91,14 +89,14 @@ public class TestBuiltinFunctions extends QueryTestCaseBase { cleanupQuery(res); } - @Test - public void testRandom() throws Exception { - ResultSet res = executeQuery(); - while(res.next()) { - assertTrue(res.getInt(2) >= 0 && res.getInt(2) < 3); - } - cleanupQuery(res); - } +// @Test +// public void testRandom() throws Exception { +// ResultSet res = executeQuery(); +// while(res.next()) { +// assertTrue(res.getInt(2) >= 0 && res.getInt(2) < 3); +// } +// cleanupQuery(res); +// } @Test public void testSplitPart() throws Exception { http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java index 9639bb6..1c59770 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestDateTimeFunctions.java @@ -28,6 +28,9 @@ import org.joda.time.DateTime; import org.junit.Test; import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.TimeZone; import static org.apache.tajo.common.TajoDataTypes.Type.*; @@ -372,4 +375,53 @@ public class TestDateTimeFunctions extends ExprTestBase { testSimpleEval("SELECT add_days(timestamp '2013-12-05 12:10:20', -7::INT8);", new String[]{"2013-11-28 12:10:20" + getUserTimeZoneDisplay()}); } + + @Test + public void testDateTimeNow() throws IOException { + TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6")); + TimeZone systemOriginTimeZone = TimeZone.getDefault(); + TimeZone.setDefault(TimeZone.getTimeZone("GMT-6")); + try { + Date expectedDate = new Date(System.currentTimeMillis()); + + testSimpleEval("select to_char(now(), 'yyyy-MM-dd');", + new String[]{dateFormat(expectedDate, "yyyy-MM-dd")}); + testSimpleEval("select cast(extract(year from now()) as INT4);", + new String[]{dateFormat(expectedDate, "yyyy")}); + testSimpleEval("select current_date();", + new String[]{dateFormat(expectedDate, "yyyy-MM-dd")}); + testSimpleEval("select cast(extract(hour from current_time()) as INT4);", + new String[]{dateFormat(expectedDate, "HH")}); + } finally { + TajoConf.setCurrentTimeZone(originTimeZone); + TimeZone.setDefault(systemOriginTimeZone); + } + } + + @Test + public void testTimeValueKeyword() throws IOException { + TimeZone originTimeZone = TajoConf.setCurrentTimeZone(TimeZone.getTimeZone("GMT-6")); + TimeZone systemOriginTimeZone = TimeZone.getDefault(); + TimeZone.setDefault(TimeZone.getTimeZone("GMT-6")); + try { + Date expectedDate = new Date(System.currentTimeMillis()); + + testSimpleEval("select to_char(current_timestamp, 'yyyy-MM-dd');", + new String[]{dateFormat(expectedDate, "yyyy-MM-dd")}); + testSimpleEval("select cast(extract(year from current_timestamp) as INT4);", + new String[]{dateFormat(expectedDate, "yyyy")}); + testSimpleEval("select current_date;", + new String[]{dateFormat(expectedDate, "yyyy-MM-dd")}); + testSimpleEval("select cast(extract(hour from current_time) as INT4);", + new String[]{dateFormat(expectedDate, "HH")}); + } finally { + TajoConf.setCurrentTimeZone(originTimeZone); + TimeZone.setDefault(systemOriginTimeZone); + } + } + + private String dateFormat(Date date, String format) { + SimpleDateFormat df = new SimpleDateFormat(format); + return df.format(date); + } } http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java index 0655700..adc44e3 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java @@ -22,11 +22,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.tajo.IntegrationTest; import org.apache.tajo.QueryTestCaseBase; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.catalog.TableDesc; -import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.catalog.*; import org.apache.tajo.catalog.partition.PartitionMethodDesc; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.storage.StorageUtil; @@ -37,9 +33,7 @@ import org.junit.experimental.categories.Category; import java.sql.ResultSet; import java.util.List; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import static org.junit.Assert.*; @Category(IntegrationTest.class) public class TestCreateTable extends QueryTestCaseBase { @@ -59,6 +53,7 @@ public class TestCreateTable extends QueryTestCaseBase { public final void testCreateTable1() throws Exception { List createdNames = executeDDL("table1_ddl.sql", "table1", "table1"); assertTableExists(createdNames.get(0)); + executeString("DROP TABLE table1"); } @Test @@ -160,17 +155,19 @@ public class TestCreateTable extends QueryTestCaseBase { @Test public final void testDropTableIfExists() throws Exception { - executeString("CREATE DATABASE D4;").close(); + executeString("CREATE DATABASE D7;").close(); + + assertTableNotExists("d7.table1"); + executeString("CREATE TABLE d7.table1 (age int);").close(); + assertTableExists("d7.table1"); - assertTableNotExists("d4.table1"); - executeString("CREATE TABLE d4.table1 (age int);").close(); - assertTableExists("d4.table1"); + executeString("DROP TABLE d7.table1;").close(); + assertTableNotExists("d7.table1"); - executeString("DROP TABLE d4.table1;").close(); - assertTableNotExists("d4.table1"); + executeString("DROP TABLE IF EXISTS d7.table1"); + assertTableNotExists("d7.table1"); - executeString("DROP TABLE IF EXISTS d4.table1"); - assertTableNotExists("d4.table1"); + executeString("DROP DATABASE D7;").close(); } @Test http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java index 890159c..822bf51 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestInsertQuery.java @@ -149,6 +149,7 @@ public class TestInsertQuery extends QueryTestCaseBase { if (!testingCluster.isHCatalogStoreRunning()) { assertEquals(2, desc.getStats().getNumRows().intValue()); } + executeString("DROP TABLE full_table_csv PURGE"); } @Test @@ -176,6 +177,7 @@ public class TestInsertQuery extends QueryTestCaseBase { if (!testingCluster.isHCatalogStoreRunning()) { assertEquals(2, orderKeys.getStats().getNumRows().intValue()); } + executeString("DROP TABLE " + tableName + " PURGE"); } @Test @@ -193,6 +195,7 @@ public class TestInsertQuery extends QueryTestCaseBase { if (!testingCluster.isHCatalogStoreRunning()) { assertEquals(2, desc.getStats().getNumRows().intValue()); } + executeString("DROP TABLE " + tableName + " PURGE"); } @Test @@ -228,6 +231,7 @@ public class TestInsertQuery extends QueryTestCaseBase { CompressionCodec codec = factory.getCodec(file.getPath()); assertTrue(codec instanceof DeflateCodec); } + executeString("DROP TABLE " + tableName + " PURGE"); } @Test @@ -271,7 +275,7 @@ public class TestInsertQuery extends QueryTestCaseBase { res = executeString("select l_orderkey, l_partkey from full_table_parquet;"); assertResultSet(res, "testInsertOverwriteWithAsteriskUsingParquet2.result"); - executeString("DROP TABLE full_table_parquet_ddl PURGE"); + executeString("DROP TABLE full_table_parquet PURGE"); } } @@ -290,6 +294,7 @@ public class TestInsertQuery extends QueryTestCaseBase { if (!testingCluster.isHCatalogStoreRunning()) { assertEquals(5, desc.getStats().getNumRows().intValue()); } + executeString("DROP TABLE table1 PURGE"); } @Test @@ -319,6 +324,7 @@ public class TestInsertQuery extends QueryTestCaseBase { assertEquals("test", res.getString(3)); res.close(); + executeString("DROP TABLE " + tableName + " PURGE"); } @Test @@ -347,6 +353,7 @@ public class TestInsertQuery extends QueryTestCaseBase { assertEquals("test", res.getString(3)); res.close(); + executeString("DROP TABLE " + tableName + " PURGE"); } @Test http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/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 f5f98a5..1581372 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 @@ -290,6 +290,9 @@ public class TestJoinBroadcast extends QueryTestCaseBase { ResultSet res = executeQuery(); assertResultSet(res); cleanupQuery(res); + executeString("DROP TABLE JOINS.part_ PURGE"); + executeString("DROP TABLE JOINS.supplier_ PURGE"); + executeString("DROP DATABASE JOINS"); } private MasterPlan getQueryPlan(QueryId queryId) { http://git-wip-us.apache.org/repos/asf/tajo/blob/26dc6e92/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java ---------------------------------------------------------------------- diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java index 9bedc10..11cf344 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestJoinQuery.java @@ -321,6 +321,10 @@ public class TestJoinQuery extends QueryTestCaseBase { ResultSet res = executeQuery(); assertResultSet(res); cleanupQuery(res); + + executeString("DROP TABLE JOINS.part_ PURGE"); + executeString("DROP TABLE JOINS.supplier_ PURGE"); + executeString("DROP DATABASE JOINS"); } @Test @@ -360,5 +364,9 @@ public class TestJoinQuery extends QueryTestCaseBase { ResultSet res = executeJsonQuery(); assertResultSet(res); cleanupQuery(res); + + executeString("DROP TABLE JOINS.part_ PURGE"); + executeString("DROP TABLE JOINS.supplier_ PURGE"); + executeString("DROP DATABASE JOINS"); } }