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 123A710E7F for ; Mon, 31 Mar 2014 15:20:22 +0000 (UTC) Received: (qmail 74343 invoked by uid 500); 31 Mar 2014 15:20:21 -0000 Delivered-To: apmail-tajo-commits-archive@tajo.apache.org Received: (qmail 74262 invoked by uid 500); 31 Mar 2014 15:20:20 -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 74225 invoked by uid 99); 31 Mar 2014 15:20:16 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 31 Mar 2014 15:20:16 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 92DC69083FB; Mon, 31 Mar 2014 15:20:16 +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 Date: Mon, 31 Mar 2014 15:20:16 -0000 Message-Id: <9bc186852f074a108d813867a041ab68@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/2] TAJO-480: Umbrella Jira for adding ALTER TABLE statement. (Alvin Henrick via hyunsik) Repository: tajo Updated Branches: refs/heads/master 40851e565 -> bd418a5c3 http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java index 30d4c2e..35ad4c2 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/parser/SQLAnalyzer.java @@ -125,7 +125,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { Expr current = visitNon_join_query_primary(ctx.non_join_query_primary()); Expr left; - for (int i = 1; i < ctx.getChildCount();) { + for (int i = 1; i < ctx.getChildCount(); ) { int idx = i; boolean distinct = true; @@ -147,7 +147,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { left = current; current = new SetOperation(OpType.Intersect, left, right, distinct); - i+=idx; + i += idx; } } @@ -215,6 +215,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { * : select_sublist (COMMA select_sublist)* * ; * + * * @param ctx * @return */ @@ -237,6 +238,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { * | asterisked_qualifier * ; * + * * @param ctx * @return */ @@ -251,7 +253,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { @Override public RelationList visitFrom_clause(SQLParser.From_clauseContext ctx) { - Expr [] relations = new Expr[ctx.table_reference_list().table_reference().size()]; + Expr[] relations = new Expr[ctx.table_reference_list().table_reference().size()]; for (int i = 0; i < relations.length; i++) { relations[i] = visitTable_reference(ctx.table_reference_list().table_reference(i)); } @@ -308,7 +310,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { @Override public Sort visitOrderby_clause(SQLParser.Orderby_clauseContext ctx) { int size = ctx.sort_specifier_list().sort_specifier().size(); - Sort.SortSpec specs [] = new Sort.SortSpec[size]; + Sort.SortSpec specs[] = new Sort.SortSpec[size]; for (int i = 0; i < size; i++) { SQLParser.Sort_specifierContext specContext = ctx.sort_specifier_list().sort_specifier(i); Expr column = visitRow_value_predicand(specContext.key); @@ -381,7 +383,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { join_condition().search_condition()); join.setQual(searchCondition); } else if (ctx.join_specification().named_columns_join() != null) { - ColumnReferenceExpr [] columns = getColumnReferences(ctx.join_specification(). + ColumnReferenceExpr[] columns = getColumnReferences(ctx.join_specification(). named_columns_join().column_reference_list()); join.setJoinColumns(columns); } @@ -392,7 +394,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return join; } - private Expr [] getRowValuePredicandsFromOrdinaryGroupingSetList(Ordinary_grouping_set_listContext ctx) { + private Expr[] getRowValuePredicandsFromOrdinaryGroupingSetList(Ordinary_grouping_set_listContext ctx) { ArrayList rowValuePredicands = new ArrayList(); for (int i = 0; i < ctx.ordinary_grouping_set().size(); i++) { Collections.addAll(rowValuePredicands, getRowValuePredicandsFromOrdinaryGroupingSet(ctx.ordinary_grouping_set(i))); @@ -400,7 +402,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return rowValuePredicands.toArray(new Expr[rowValuePredicands.size()]); } - private Expr [] getRowValuePredicandsFromOrdinaryGroupingSet(Ordinary_grouping_setContext ctx) { + private Expr[] getRowValuePredicandsFromOrdinaryGroupingSet(Ordinary_grouping_setContext ctx) { ArrayList rowValuePredicands = new ArrayList(); if (ctx.row_value_predicand() != null) { rowValuePredicands.add(visitRow_value_predicand(ctx.row_value_predicand())); @@ -411,16 +413,16 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return rowValuePredicands.toArray(new Expr[rowValuePredicands.size()]); } - private Expr [] getRowValuePredicands(Row_value_predicand_listContext ctx) { - Expr [] rowValuePredicands = new Expr[ctx.row_value_predicand().size()]; + private Expr[] getRowValuePredicands(Row_value_predicand_listContext ctx) { + Expr[] rowValuePredicands = new Expr[ctx.row_value_predicand().size()]; for (int i = 0; i < rowValuePredicands.length; i++) { rowValuePredicands[i] = visitRow_value_predicand(ctx.row_value_predicand(i)); } return rowValuePredicands; } - private ColumnReferenceExpr [] getColumnReferences(Column_reference_listContext ctx) { - ColumnReferenceExpr [] columnRefs = new ColumnReferenceExpr[ctx.column_reference().size()]; + private ColumnReferenceExpr[] getColumnReferences(Column_reference_listContext ctx) { + ColumnReferenceExpr[] columnRefs = new ColumnReferenceExpr[ctx.column_reference().size()]; for (int i = 0; i < columnRefs.length; i++) { columnRefs[i] = visitColumn_reference(ctx.column_reference(i)); } @@ -482,7 +484,8 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { } } - @Override public CaseWhenPredicate visitSearched_case(SQLParser.Searched_caseContext ctx) { + @Override + public CaseWhenPredicate visitSearched_case(SQLParser.Searched_caseContext ctx) { CaseWhenPredicate caseWhen = new CaseWhenPredicate(); for (int i = 0; i < ctx.searched_when_clause().size(); i++) { @@ -497,7 +500,8 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return caseWhen; } - @Override public Expr visitCommon_value_expression(SQLParser.Common_value_expressionContext ctx) { + @Override + public Expr visitCommon_value_expression(SQLParser.Common_value_expressionContext ctx) { if (checkIfExist(ctx.NULL())) { return new NullLiteral(); } else { @@ -671,7 +675,8 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return current; } - @Override public Expr visitFactor(SQLParser.FactorContext ctx) { + @Override + public Expr visitFactor(SQLParser.FactorContext ctx) { Expr current = visitNumeric_primary(ctx.numeric_primary()); if (checkIfExist(ctx.sign()) && checkIfExist(ctx.sign().MINUS())) { current = new SignedExpr(true, current); @@ -696,27 +701,44 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { public static OpType tokenToExprType(int tokenId) { switch (tokenId) { - case SQLParser.UNION: return OpType.Union; - case SQLParser.EXCEPT: return OpType.Except; - case SQLParser.INTERSECT: return OpType.Intersect; - - case SQLParser.AND: return OpType.And; - case SQLParser.OR: return OpType.Or; - - case SQLParser.EQUAL: return OpType.Equals; - case SQLParser.NOT_EQUAL: return OpType.NotEquals; - case SQLParser.LTH: return OpType.LessThan; - case SQLParser.LEQ: return OpType.LessThanOrEquals; - case SQLParser.GTH: return OpType.GreaterThan; - case SQLParser.GEQ: return OpType.GreaterThanOrEquals; - - case SQLParser.MULTIPLY: return OpType.Multiply; - case SQLParser.DIVIDE: return OpType.Divide; - case SQLParser.MODULAR: return OpType.Modular; - case SQLParser.PLUS: return OpType.Plus; - case SQLParser.MINUS: return OpType.Minus; - - default: throw new RuntimeException("Unknown Token Id: " + tokenId); + case SQLParser.UNION: + return OpType.Union; + case SQLParser.EXCEPT: + return OpType.Except; + case SQLParser.INTERSECT: + return OpType.Intersect; + + case SQLParser.AND: + return OpType.And; + case SQLParser.OR: + return OpType.Or; + + case SQLParser.EQUAL: + return OpType.Equals; + case SQLParser.NOT_EQUAL: + return OpType.NotEquals; + case SQLParser.LTH: + return OpType.LessThan; + case SQLParser.LEQ: + return OpType.LessThanOrEquals; + case SQLParser.GTH: + return OpType.GreaterThan; + case SQLParser.GEQ: + return OpType.GreaterThanOrEquals; + + case SQLParser.MULTIPLY: + return OpType.Multiply; + case SQLParser.DIVIDE: + return OpType.Divide; + case SQLParser.MODULAR: + return OpType.Modular; + case SQLParser.PLUS: + return OpType.Plus; + case SQLParser.MINUS: + return OpType.Minus; + + default: + throw new RuntimeException("Unknown Token Id: " + tokenId); } } @@ -730,7 +752,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { public Expr visitIn_predicate_value(SQLParser.In_predicate_valueContext ctx) { if (checkIfExist(ctx.in_value_list())) { int size = ctx.in_value_list().row_value_expression().size(); - Expr [] exprs = new Expr[size]; + Expr[] exprs = new Expr[size]; for (int i = 0; i < size; i++) { exprs[i] = visitRow_value_expression(ctx.in_value_list().row_value_expression(i)); } @@ -743,7 +765,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { @Override public Expr visitArray(SQLParser.ArrayContext ctx) { int size = ctx.numeric_value_expression().size(); - Expr [] exprs = new Expr[size]; + Expr[] exprs = new Expr[size]; for (int i = 0; i < size; i++) { exprs[i] = visit(ctx.numeric_value_expression(i)); } @@ -825,7 +847,8 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { } } - @Override public FunctionExpr visitAggregate_function( SQLParser.Aggregate_functionContext ctx) { + @Override + public FunctionExpr visitAggregate_function(SQLParser.Aggregate_functionContext ctx) { if (ctx.COUNT() != null && ctx.MULTIPLY() != null) { return new CountRowsFunctionExpr(); } else { @@ -833,7 +856,8 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { } } - @Override public FunctionExpr visitGeneral_set_function(SQLParser.General_set_functionContext ctx) { + @Override + public FunctionExpr visitGeneral_set_function(SQLParser.General_set_functionContext ctx) { String signature = ctx.set_function_type().getText(); boolean distinct = checkIfExist(ctx.set_qualifier()) && checkIfExist(ctx.set_qualifier().DISTINCT()); Expr param = visitValue_expression(ctx.value_expression()); @@ -847,7 +871,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { FunctionExpr function = new FunctionExpr(signature); if (ctx.sql_argument_list() != null) { int numArgs = ctx.sql_argument_list().value_expression().size(); - Expr [] argument_list = new Expr[numArgs]; + Expr[] argument_list = new Expr[numArgs]; for (int i = 0; i < numArgs; i++) { argument_list[i] = visitValue_expression(ctx.sql_argument_list(). value_expression().get(i)); @@ -894,7 +918,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { right = visitCharacter_factor((Character_factorContext) ctx.getChild(i)); if (left.getType() == OpType.Literal && right.getType() == OpType.Literal) { - current = new LiteralValue(((LiteralValue)left).getValue() + ((LiteralValue)right).getValue(), + current = new LiteralValue(((LiteralValue) left).getValue() + ((LiteralValue) right).getValue(), LiteralType.String); } else { current = new BinaryOperator(OpType.Concatenate, left, right); @@ -927,7 +951,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { } String functionName = "date_part"; - Expr [] params = new Expr[] {extractTarget, extractSource}; + Expr[] params = new Expr[]{extractTarget, extractSource}; return new FunctionExpr(functionName, params); } @@ -954,11 +978,11 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { trimCharacters = visitCharacter_value_expression(ctx.trim_operands().trim_character); } - Expr [] params; + Expr[] params; if (trimCharacters != null) { - params = new Expr[] {trimSource, trimCharacters}; + params = new Expr[]{trimSource, trimCharacters}; } else { - params = new Expr[] {trimSource}; + params = new Expr[]{trimSource}; } return new FunctionExpr(functionName, params); @@ -982,7 +1006,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { if (checkIfExist(ctx.EXTERNAL())) { createTable.setExternal(); - CreateTable.ColumnDefinition [] elements = getDefinitions(ctx.table_elements()); + ColumnDefinition[] elements = getDefinitions(ctx.table_elements()); String fileType = ctx.file_type.getText(); String path = stripQuote(ctx.path.getText()); @@ -991,7 +1015,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { createTable.setLocation(path); } else { if (checkIfExist(ctx.table_elements())) { - CreateTable.ColumnDefinition [] elements = getDefinitions(ctx.table_elements()); + ColumnDefinition[] elements = getDefinitions(ctx.table_elements()); createTable.setTableElements(elements); } @@ -1019,13 +1043,13 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return createTable; } - private CreateTable.ColumnDefinition [] getDefinitions(SQLParser.Table_elementsContext ctx) { + private ColumnDefinition[] getDefinitions(SQLParser.Table_elementsContext ctx) { int size = ctx.field_element().size(); - CreateTable.ColumnDefinition [] elements = new CreateTable.ColumnDefinition[size]; + ColumnDefinition[] elements = new ColumnDefinition[size]; for (int i = 0; i < size; i++) { String name = ctx.field_element(i).name.getText(); DataTypeExpr typeDef = visitData_type(ctx.field_element(i).field_type().data_type()); - elements[i] = new CreateTable.ColumnDefinition(name, typeDef); + elements[i] = new ColumnDefinition(name, typeDef); } return elements; @@ -1074,7 +1098,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { for (List_value_partitionContext listValuePartition : partitions) { int size = listValuePartition.in_value_list().row_value_expression().size(); - Expr [] exprs = new Expr[size]; + Expr[] exprs = new Expr[size]; for (int i = 0; i < size; i++) { exprs[i] = visitRow_value_expression(listValuePartition.in_value_list().row_value_expression(i)); } @@ -1120,7 +1144,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { } } else if (character_string_type.TEXT() != null) { - typeDefinition = new DataTypeExpr(Type.TEXT.name()); + typeDefinition = new DataTypeExpr(Type.TEXT.name()); } } else if (predefined_type.national_character_string_type() != null) { @@ -1187,7 +1211,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { typeDefinition = new DataTypeExpr(Type.FLOAT8.name()); } } - } else if (predefined_type.boolean_type() != null) { + } else if (predefined_type.boolean_type() != null) { typeDefinition = new DataTypeExpr(Type.BOOLEAN.name()); } else if (predefined_type.datetime_type() != null) { SQLParser.Datetime_typeContext dateTimeType = predefined_type.datetime_type(); @@ -1246,7 +1270,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { insertExpr.setTableName(ctx.table_name().getText()); if (ctx.column_name_list() != null) { - String [] targetColumns = new String[ctx.column_name_list().identifier().size()]; + String[] targetColumns = new String[ctx.column_name_list().identifier().size()]; for (int i = 0; i < targetColumns.length; i++) { targetColumns[i] = ctx.column_name_list().identifier().get(i).getText(); } @@ -1296,7 +1320,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { for (Map.Entry entry : map.entrySet()) { if (entry.getKey().equals(CatalogConstants.CSVFILE_DELIMITER)) { params.put(entry.getKey(), escapeDelimiter(entry.getValue())); - } else { + } else { params.put(entry.getKey(), entry.getValue()); } } @@ -1317,13 +1341,15 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { return str.substring(1, str.length() - 1); } - @Override public Expr visitCast_specification(SQLParser.Cast_specificationContext ctx) { + @Override + public Expr visitCast_specification(SQLParser.Cast_specificationContext ctx) { Expr operand = visitChildren(ctx.cast_operand()); DataTypeExpr castTarget = visitData_type(ctx.cast_target().data_type()); return new CastExpr(operand, castTarget); } - @Override public Expr visitUnsigned_value_specification(@NotNull SQLParser.Unsigned_value_specificationContext ctx) { + @Override + public Expr visitUnsigned_value_specification(@NotNull SQLParser.Unsigned_value_specificationContext ctx) { return visitChildren(ctx); } @@ -1351,7 +1377,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { public Expr visitDatetime_literal(@NotNull SQLParser.Datetime_literalContext ctx) { if (checkIfExist(ctx.time_literal())) { return visitTime_literal(ctx.time_literal()); - } else if(checkIfExist(ctx.date_literal())) { + } else if (checkIfExist(ctx.date_literal())) { return visitDate_literal(ctx.date_literal()); } else { return visitTimestamp_literal(ctx.timestamp_literal()); @@ -1373,7 +1399,7 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { @Override public Expr visitTimestamp_literal(SQLParser.Timestamp_literalContext ctx) { String timestampStr = stripQuote(ctx.timestamp_string.getText()); - String [] parts = timestampStr.split(" "); + String[] parts = timestampStr.split(" "); String datePart = parts[0]; String timePart = parts[1]; return new TimestampLiteral(parseDate(datePart), parseTime(timePart)); @@ -1381,18 +1407,18 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { private DateValue parseDate(String datePart) { // e.g., 1980-04-01 - String [] parts = datePart.split("-"); + String[] parts = datePart.split("-"); return new DateValue(parts[0], parts[1], parts[2]); } private TimeValue parseTime(String timePart) { // e.g., 12:01:50.399 - String [] parts = timePart.split(":"); + String[] parts = timePart.split(":"); TimeValue time; boolean hasFractionOfSeconds = parts[2].indexOf('.') > 0; if (hasFractionOfSeconds) { - String [] secondsParts = parts[2].split("\\."); + String[] secondsParts = parts[2].split("\\."); time = new TimeValue(parts[0], parts[1], secondsParts[0]); if (secondsParts.length == 2) { time.setSecondsFraction(secondsParts[1]); @@ -1402,4 +1428,77 @@ public class SQLAnalyzer extends SQLParserBaseVisitor { } return time; } + + @Override + public Expr visitAlter_table_statement(SQLParser.Alter_table_statementContext ctx) { + + final List tables = ctx.table_name(); + + final AlterTable alterTable = new AlterTable(tables.get(0).getText()); + + if (tables.size() == 2) { + alterTable.setNewTableName(tables.get(1).getText()); + } + + if (checkIfExist(ctx.column_name()) && ctx.column_name().size() == 2) { + final List columns = ctx.column_name(); + alterTable.setColumnName(columns.get(0).getText()); + alterTable.setNewColumnName(columns.get(1).getText()); + } + + Field_elementContext field_elementContext = ctx.field_element(); + if (checkIfExist(field_elementContext)) { + final String name = field_elementContext.name.getText(); + final DataTypeExpr typeDef = visitData_type(field_elementContext.field_type().data_type()); + final ColumnDefinition columnDefinition = new ColumnDefinition(name, typeDef); + alterTable.setAddNewColumn(columnDefinition); + } + + alterTable.setAlterTableOpType(determineAlterTableType(ctx)); + + return alterTable; + } + + private AlterTableOpType determineAlterTableType(SQLParser.Alter_table_statementContext ctx) { + + final int RENAME_MASK = 00000001; + final int COLUMN_MASK = 00000010; + final int TO_MASK = 00000100; + final int ADD_MASK = 00001000; + + int val = 00000000; + + for (int idx = 1; idx < ctx.getChildCount(); idx++) { + + if (ctx.getChild(idx) instanceof TerminalNode) { + if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == RENAME) { + val = val | RENAME_MASK; + } + if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == COLUMN) { + val = val | COLUMN_MASK; + } + if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == TO) { + val = val | TO_MASK; + } + if (((TerminalNode) ctx.getChild(idx)).getSymbol().getType() == ADD) { + val = val | ADD_MASK; + } + } + } + return evaluateAlterTableOperationTye(val); + } + + private AlterTableOpType evaluateAlterTableOperationTye(final int value) { + + switch (value) { + case 65: + return AlterTableOpType.RENAME_TABLE; + case 73: + return AlterTableOpType.RENAME_COLUMN; + case 520: + return AlterTableOpType.ADD_COLUMN; + default: + return null; + } + } } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java index 84cdc08..0ea2c77 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/AlgebraVisitor.java @@ -45,8 +45,9 @@ public interface AlgebraVisitor { RESULT visitDropDatabase(CONTEXT ctx, Stack stack, DropDatabase expr) throws PlanningException; RESULT visitCreateTable(CONTEXT ctx, Stack stack, CreateTable expr) throws PlanningException; RESULT visitDropTable(CONTEXT ctx, Stack stack, DropTable expr) throws PlanningException; + RESULT visitAlterTable(CONTEXT ctx, Stack stack, AlterTable expr) throws PlanningException; - // Insert or Update + // Insert or Update RESULT visitInsert(CONTEXT ctx, Stack stack, Insert expr) throws PlanningException; // Logical operators http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java index 62cee57..6f217a7 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BaseAlgebraVisitor.java @@ -109,6 +109,9 @@ public class BaseAlgebraVisitor implements AlgebraVisitor implements AlgebraVisitor stack, AlterTable expr) throws PlanningException { + return null; + } + /////////////////////////////////////////////////////////////////////////////////////////////////////////// // Insert or Update Section /////////////////////////////////////////////////////////////////////////////////////////////////////////// http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java index 772e5fb..d569758 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/BasicLogicalPlanVisitor.java @@ -116,6 +116,9 @@ public class BasicLogicalPlanVisitor implements LogicalPlanVisi case DROP_TABLE: current = visitDropTable(context, plan, block, (DropTableNode) node, stack); break; + case ALTER_TABLE: + current = visitAlterTable(context, plan, block, (AlterTableNode) node, stack); + break; default: throw new PlanningException("Unknown logical node type: " + node.getType()); } @@ -298,4 +301,9 @@ public class BasicLogicalPlanVisitor implements LogicalPlanVisi Stack stack) { return null; } + @Override + public RESULT visitAlterTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, AlterTableNode node, + Stack stack) { + return null; + } } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java index 1ac416f..540d45b 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanPreprocessor.java @@ -413,6 +413,13 @@ class LogicalPlanPreprocessor extends BaseAlgebraVisitor stack, AlterTable expr) + throws PlanningException { + AlterTableNode alterTableNode = ctx.plan.createNode(AlterTableNode.class); + return alterTableNode; + } + /////////////////////////////////////////////////////////////////////////////////////////////////////////// // Insert or Update Section /////////////////////////////////////////////////////////////////////////////////////////////////////////// http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java index 76454b9..7f38026 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanVisitor.java @@ -82,4 +82,7 @@ public interface LogicalPlanVisitor { RESULT visitDropTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, DropTableNode node, Stack stack) throws PlanningException; + + RESULT visitAlterTable(CONTEXT context, LogicalPlan plan, LogicalPlan.QueryBlock block, AlterTableNode node, + Stack stack) throws PlanningException; } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java index 15fe6c0..317c05b 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/LogicalPlanner.java @@ -30,7 +30,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.tajo.algebra.*; -import org.apache.tajo.algebra.CreateTable.ColumnDefinition; import org.apache.tajo.catalog.*; import org.apache.tajo.catalog.partition.PartitionMethodDesc; import org.apache.tajo.catalog.proto.CatalogProtos; @@ -1412,10 +1411,10 @@ public class LogicalPlanner extends BaseAlgebraVisitor stack, AlterTable alterTable) { + AlterTableNode alterTableNode = context.queryBlock.getNodeFromExpr(alterTable); + alterTableNode.setTableName(alterTable.getTableName()); + alterTableNode.setNewTableName(alterTable.getNewTableName()); + alterTableNode.setColumnName(alterTable.getColumnName()); + alterTableNode.setNewColumnName(alterTable.getNewColumnName()); + + if (null != alterTable.getAddNewColumn()) { + alterTableNode.setAddNewColumn(convertColumn(alterTable.getAddNewColumn())); + } + alterTableNode.setAlterTableOpType(alterTable.getAlterTableOpType()); + return alterTableNode; + } + /*=============================================================================================== Util SECTION ===============================================================================================*/ http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java index fbd65a9..ccd9847 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/PlannerUtil.java @@ -49,9 +49,9 @@ public class PlannerUtil { return type == NodeType.CREATE_DATABASE || - type == NodeType.DROP_DATABASE || - (type == NodeType.CREATE_TABLE && !((CreateTableNode)baseNode).hasSubQuery()) || - baseNode.getType() == NodeType.DROP_TABLE; + type == NodeType.DROP_DATABASE || + (type == NodeType.CREATE_TABLE && !((CreateTableNode) baseNode).hasSubQuery()) || + baseNode.getType() == NodeType.DROP_TABLE || baseNode.getType() == NodeType.ALTER_TABLE; } /** @@ -60,9 +60,9 @@ public class PlannerUtil { * @param from The LogicalNode to start visiting LogicalNodes. * @return an array of all descendant RelationNode of LogicalNode. */ - public static String [] getRelationLineage(LogicalNode from) { - LogicalNode [] scans = findAllNodes(from, NodeType.SCAN, NodeType.PARTITIONS_SCAN); - String [] tableNames = new String[scans.length]; + public static String[] getRelationLineage(LogicalNode from) { + LogicalNode[] scans = findAllNodes(from, NodeType.SCAN, NodeType.PARTITIONS_SCAN); + String[] tableNames = new String[scans.length]; ScanNode scan; for (int i = 0; i < scans.length; i++) { scan = (ScanNode) scans[i]; @@ -106,11 +106,11 @@ public class PlannerUtil { return node; } } - + /** * Delete the logical node from a plan. * - * @param parent this node must be a parent node of one node to be removed. + * @param parent this node must be a parent node of one node to be removed. * @param tobeRemoved this node must be a child node of the parent. */ public static LogicalNode deleteNode(LogicalNode parent, LogicalNode tobeRemoved) { @@ -137,7 +137,7 @@ public class PlannerUtil { } } else { throw new InvalidQueryException("Unexpected logical plan: " + parent); - } + } return child; } @@ -233,13 +233,13 @@ public class PlannerUtil { @Override public LogicalNode visitPartitionedTableScan(ReplacerContext context, LogicalPlan plan, LogicalPlan. - QueryBlock block,PartitionedTableScanNode node, Stack stack) + QueryBlock block, PartitionedTableScanNode node, Stack stack) throws PlanningException { return node; } } - + public static void replaceNode(LogicalNode plan, LogicalNode newNode, NodeType type) { LogicalNode parent = findTopParentNode(plan, type); Preconditions.checkArgument(parent instanceof UnaryNode); @@ -254,7 +254,7 @@ public class PlannerUtil { /** * Find the top logical node matched to type from the given node - * + * * @param node start node * @param type to find * @return a found logical node @@ -262,10 +262,10 @@ public class PlannerUtil { public static T findTopNode(LogicalNode node, NodeType type) { Preconditions.checkNotNull(node); Preconditions.checkNotNull(type); - + LogicalNodeFinder finder = new LogicalNodeFinder(type); node.preOrder(finder); - + if (finder.getFoundNodes().size() == 0) { return null; } @@ -299,7 +299,7 @@ public class PlannerUtil { * @param type to find * @return a found logical node */ - public static LogicalNode [] findAllNodes(LogicalNode node, NodeType...type) { + public static LogicalNode[] findAllNodes(LogicalNode node, NodeType... type) { Preconditions.checkNotNull(node); Preconditions.checkNotNull(type); @@ -307,15 +307,15 @@ public class PlannerUtil { node.postOrder(finder); if (finder.getFoundNodes().size() == 0) { - return new LogicalNode[] {}; + return new LogicalNode[]{}; } List founds = finder.getFoundNodes(); return founds.toArray(new LogicalNode[founds.size()]); } - + /** * Find a parent node of a given-typed operator. - * + * * @param node start node * @param type to find * @return the parent node of a found logical node @@ -323,10 +323,10 @@ public class PlannerUtil { public static T findTopParentNode(LogicalNode node, NodeType type) { Preconditions.checkNotNull(node); Preconditions.checkNotNull(type); - + ParentNodeFinder finder = new ParentNodeFinder(type); node.postOrder(finder); - + if (finder.getFoundNodes().size() == 0) { return null; } @@ -339,7 +339,7 @@ public class PlannerUtil { private boolean topmost = false; private boolean finished = false; - public LogicalNodeFinder(NodeType...type) { + public LogicalNodeFinder(NodeType... type) { this.tofind = type; } @@ -366,11 +366,11 @@ public class PlannerUtil { return list; } - public LogicalNode [] getFoundNodeArray() { + public LogicalNode[] getFoundNodeArray() { return list.toArray(new LogicalNode[list.size()]); } } - + private static class ParentNodeFinder implements LogicalNodeVisitor { private List list = new ArrayList(); private NodeType tofind; @@ -386,7 +386,7 @@ public class PlannerUtil { if (unary.getChild().getType() == tofind) { list.add(node); } - } else if (node instanceof BinaryNode){ + } else if (node instanceof BinaryNode) { BinaryNode bin = (BinaryNode) node; if (bin.getLeftChild().getType() == tofind || bin.getRightChild().getType() == tofind) { @@ -403,10 +403,10 @@ public class PlannerUtil { /** * fill targets with FieldEvals from a given schema * - * @param schema to be transformed to targets + * @param schema to be transformed to targets * @param targets to be filled */ - public static void schemaToTargets(Schema schema, Target [] targets) { + public static void schemaToTargets(Schema schema, Target[] targets) { FieldEval eval; for (int i = 0; i < schema.size(); i++) { eval = new FieldEval(schema.getColumn(i)); @@ -440,7 +440,7 @@ public class PlannerUtil { return schemaToSortSpecs(schema.toArray()); } - public static SortSpec[] schemaToSortSpecs(Column [] columns) { + public static SortSpec[] schemaToSortSpecs(Column[] columns) { SortSpec[] specs = new SortSpec[columns.length]; for (int i = 0; i < columns.length; i++) { @@ -450,7 +450,7 @@ public class PlannerUtil { return specs; } - public static SortSpec [] columnsToSortSpec(Collection columns) { + public static SortSpec[] columnsToSortSpec(Collection columns) { SortSpec[] specs = new SortSpec[columns.size()]; int i = 0; for (Column column : columns) { @@ -472,7 +472,7 @@ public class PlannerUtil { /** * is it join qual or not? * - * @param qual The condition to be checked + * @param qual The condition to be checked * @return true if two operands refers to columns and the operator is comparison, */ public static boolean isJoinQual(EvalNode qual) { @@ -489,7 +489,7 @@ public class PlannerUtil { } public static SortSpec[][] getSortKeysFromJoinQual(EvalNode joinQual, Schema outer, Schema inner) { - List joinKeyPairs = getJoinKeyPairs(joinQual, outer, inner); + List joinKeyPairs = getJoinKeyPairs(joinQual, outer, inner); SortSpec[] outerSortSpec = new SortSpec[joinKeyPairs.size()]; SortSpec[] innerSortSpec = new SortSpec[joinKeyPairs.size()]; @@ -498,12 +498,12 @@ public class PlannerUtil { innerSortSpec[i] = new SortSpec(joinKeyPairs.get(i)[1]); } - return new SortSpec[][] {outerSortSpec, innerSortSpec}; + return new SortSpec[][]{outerSortSpec, innerSortSpec}; } public static TupleComparator[] getComparatorsFromJoinQual(EvalNode joinQual, Schema leftSchema, Schema rightSchema) { SortSpec[][] sortSpecs = getSortKeysFromJoinQual(joinQual, leftSchema, rightSchema); - TupleComparator [] comparators = new TupleComparator[2]; + TupleComparator[] comparators = new TupleComparator[2]; comparators[0] = new TupleComparator(leftSchema, sortSpecs[0]); comparators[1] = new TupleComparator(rightSchema, sortSpecs[1]); return comparators; @@ -512,27 +512,27 @@ public class PlannerUtil { /** * @return the first array contains left table's columns, and the second array contains right table's columns. */ - public static Column [][] joinJoinKeyForEachTable(EvalNode joinQual, Schema leftSchema, Schema rightSchema) { - List joinKeys = getJoinKeyPairs(joinQual, leftSchema, rightSchema); - Column [] leftColumns = new Column[joinKeys.size()]; - Column [] rightColumns = new Column[joinKeys.size()]; + public static Column[][] joinJoinKeyForEachTable(EvalNode joinQual, Schema leftSchema, Schema rightSchema) { + List joinKeys = getJoinKeyPairs(joinQual, leftSchema, rightSchema); + Column[] leftColumns = new Column[joinKeys.size()]; + Column[] rightColumns = new Column[joinKeys.size()]; for (int i = 0; i < joinKeys.size(); i++) { leftColumns[i] = joinKeys.get(i)[0]; rightColumns[i] = joinKeys.get(i)[1]; } - return new Column[][] {leftColumns, rightColumns}; + return new Column[][]{leftColumns, rightColumns}; } - public static List getJoinKeyPairs(EvalNode joinQual, Schema leftSchema, Schema rightSchema) { + public static List getJoinKeyPairs(EvalNode joinQual, Schema leftSchema, Schema rightSchema) { JoinKeyPairFinder finder = new JoinKeyPairFinder(leftSchema, rightSchema); joinQual.preOrder(finder); return finder.getPairs(); } public static class JoinKeyPairFinder implements EvalNodeVisitor { - private final List pairs = Lists.newArrayList(); - private Schema [] schemas = new Schema[2]; + private final List pairs = Lists.newArrayList(); + private Schema[] schemas = new Schema[2]; public JoinKeyPairFinder(Schema outer, Schema inner) { schemas[0] = outer; @@ -542,13 +542,13 @@ public class PlannerUtil { @Override public void visit(EvalNode node) { if (EvalTreeUtil.isJoinQual(node)) { - Column [] pair = new Column[2]; + Column[] pair = new Column[2]; for (int i = 0; i <= 1; i++) { // access left, right sub expression Column column = EvalTreeUtil.findAllColumnRefs(node.getExpr(i)).get(0); for (int j = 0; j < schemas.length; j++) { - // check whether the column is for either outer or inner - // 0 is outer, and 1 is inner + // check whether the column is for either outer or inner + // 0 is outer, and 1 is inner if (schemas[j].containsByQualifiedName(column.getQualifiedName())) { pair[j] = column; } @@ -562,7 +562,7 @@ public class PlannerUtil { } } - public List getPairs() { + public List getPairs() { return this.pairs; } } @@ -573,7 +573,7 @@ public class PlannerUtil { public static Schema targetToSchema(Target[] targets) { Schema schema = new Schema(); - for(Target t : targets) { + for (Target t : targets) { DataType type = t.getEvalTree().getValueType(); String name; if (t.hasAlias()) { @@ -595,9 +595,9 @@ public class PlannerUtil { * @param sourceTargets The targets to be stripped * @return The stripped targets */ - public static Target [] stripTarget(Target [] sourceTargets) { - Target [] copy = new Target[sourceTargets.length]; - for(int i = 0; i < sourceTargets.length; i++) { + public static Target[] stripTarget(Target[] sourceTargets) { + Target[] copy = new Target[sourceTargets.length]; + for (int i = 0; i < sourceTargets.length; i++) { try { copy[i] = (Target) sourceTargets[i].clone(); } catch (CloneNotSupportedException e) { @@ -650,6 +650,7 @@ public class PlannerUtil { boolean generalSetFunction; boolean distinctSetFunction; } + static class AggregationFunctionFinder extends SimpleAlgebraVisitor { @Override public Object visitCountRowsFunction(AggFunctionFoundResult ctx, Stack stack, CountRowsFunctionExpr expr) @@ -670,7 +671,7 @@ public class PlannerUtil { public static Collection toQualifiedFieldNames(Collection fieldNames, String qualifier) { List names = TUtil.newList(); for (String n : fieldNames) { - String [] parts = n.split("\\."); + String[] parts = n.split("\\."); if (parts.length == 1) { names.add(qualifier + "." + parts[0]); } else { @@ -680,8 +681,8 @@ public class PlannerUtil { return names; } - public static SortSpec [] convertSortSpecs(Collection sortSpecProtos) { - SortSpec [] sortSpecs = new SortSpec[sortSpecProtos.size()]; + public static SortSpec[] convertSortSpecs(Collection sortSpecProtos) { + SortSpec[] sortSpecs = new SortSpec[sortSpecProtos.size()]; int i = 0; for (CatalogProtos.SortSpecProto proto : sortSpecProtos) { sortSpecs[i++] = new SortSpec(proto); @@ -701,7 +702,7 @@ public class PlannerUtil { StringBuilder explains = new StringBuilder(); try { ExplainLogicalPlanVisitor.Context explainContext = explain.getBlockPlanStrings(null, node); - while(!explainContext.explains.empty()) { + while (!explainContext.explains.empty()) { explains.append( ExplainLogicalPlanVisitor.printDepthString(explainContext.getMaxDepth(), explainContext.explains.pop())); } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java index dc7b7a2..bae6e4a 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/SimpleAlgebraVisitor.java @@ -121,10 +121,15 @@ public abstract class SimpleAlgebraVisitor extends BaseAlgebraV return super.visitDropTable(ctx, stack, expr); } + @Override + public RESULT visitAlterTable(CONTEXT ctx, Stack stack, AlterTable expr) throws PlanningException { + return super.visitAlterTable(ctx, stack, expr); + } + /////////////////////////////////////////////////////////////////////////////////////////////////////////// // Insert or Update Section /////////////////////////////////////////////////////////////////////////////////////////////////////////// - + @Override public RESULT visitInsert(CONTEXT ctx, Stack stack, Insert expr) throws PlanningException { return super.visitInsert(ctx, stack, expr); } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java new file mode 100644 index 0000000..76a47d0 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/AlterTableNode.java @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.planner.logical; + + +import com.google.gson.annotations.Expose; +import org.apache.tajo.algebra.AlterTableOpType; +import org.apache.tajo.catalog.Column; +import org.apache.tajo.engine.planner.PlanString; + +public class AlterTableNode extends LogicalNode { + + @Expose + private String tableName; + @Expose + private String newTableName; + @Expose + private String columnName; + @Expose + private String newColumnName; + @Expose + private Column addNewColumn; + @Expose + private AlterTableOpType alterTableOpType; + + public AlterTableNode(int pid) { + super(pid, NodeType.ALTER_TABLE); + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public String getNewTableName() { + return newTableName; + } + + public void setNewTableName(String newTableName) { + this.newTableName = newTableName; + } + + public String getColumnName() { + return columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public String getNewColumnName() { + return newColumnName; + } + + public void setNewColumnName(String newColumnName) { + this.newColumnName = newColumnName; + } + + public Column getAddNewColumn() { + return addNewColumn; + } + + public void setAddNewColumn(Column addNewColumn) { + this.addNewColumn = addNewColumn; + } + + public AlterTableOpType getAlterTableOpType() { + return alterTableOpType; + } + + public void setAlterTableOpType(AlterTableOpType alterTableOpType) { + this.alterTableOpType = alterTableOpType; + } + + @Override + public PlanString getPlanString() { + return new PlanString(this); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof AlterTableNode) { + AlterTableNode other = (AlterTableNode) obj; + return super.equals(other); + } else { + return false; + } + } + + /*@Override + public Object clone() throws CloneNotSupportedException { + AlterTableNode alterTableNode = (AlterTableNode) super.clone(); + alterTableNode.tableName = tableName; + alterTableNode.newTableName = newTableName; + alterTableNode.columnName = columnName; + alterTableNode.newColumnName=newColumnName; + alterTableNode.addNewColumn =(Column) addNewColumn.clone(); + return alterTableNode; + }*/ + + @Override + public String toString() { + return "AlterTable (table=" + tableName + ")"; + } + + @Override + public void preOrder(LogicalNodeVisitor visitor) { + visitor.visit(this); + } + + @Override + public void postOrder(LogicalNodeVisitor visitor) { + visitor.visit(this); + } +} http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java index 2b453fb..375926e 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/engine/planner/logical/NodeType.java @@ -21,6 +21,7 @@ */ package org.apache.tajo.engine.planner.logical; + /** * This indicates a logical node type. */ @@ -47,8 +48,8 @@ public enum NodeType { CREATE_DATABASE(CreateDatabaseNode.class), DROP_DATABASE(DropDatabaseNode.class), CREATE_TABLE(CreateTableNode.class), - DROP_TABLE(DropTableNode.class) - ; + DROP_TABLE(DropTableNode.class), + ALTER_TABLE (AlterTableNode.class); private final Class baseClass; http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java index fe3caeb..b0c0614 100644 --- a/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java +++ b/tajo-core/tajo-core-backend/src/main/java/org/apache/tajo/master/GlobalEngine.java @@ -235,7 +235,10 @@ public class GlobalEngine extends AbstractService { DropTableNode dropTable = (DropTableNode) root; dropTable(session, dropTable.getTableName(), dropTable.isIfExists(), dropTable.isPurge()); return true; - + case ALTER_TABLE: + AlterTableNode alterTable = (AlterTableNode) root; + alterTable(session,alterTable); + return true; default: throw new InternalError("updateQuery cannot handle such query: \n" + root.toJson()); } @@ -275,6 +278,82 @@ public class GlobalEngine extends AbstractService { return plan; } + /** + * Alter a given table + */ + public void alterTable(final Session session, final AlterTableNode alterTable) throws IOException { + + final CatalogService catalog = context.getCatalog(); + final String tableName = alterTable.getTableName(); + + String databaseName; + String simpleTableName; + if (CatalogUtil.isFQTableName(tableName)) { + String[] split = CatalogUtil.splitFQTableName(tableName); + databaseName = split[0]; + simpleTableName = split[1]; + } else { + databaseName = session.getCurrentDatabase(); + simpleTableName = tableName; + } + final String qualifiedName = CatalogUtil.buildFQName(databaseName, simpleTableName); + + if (!catalog.existsTable(databaseName, simpleTableName)) { + throw new NoSuchTableException(qualifiedName); + } + + switch (alterTable.getAlterTableOpType()) { + case RENAME_TABLE: + if (!catalog.existsTable(databaseName, simpleTableName)) { + throw new NoSuchTableException(alterTable.getTableName()); + } + if (catalog.existsTable(databaseName, alterTable.getNewTableName())) { + throw new AlreadyExistsTableException(alterTable.getNewTableName()); + } + + TableDesc desc = catalog.getTableDesc(databaseName, simpleTableName); + + if (!desc.isExternal()) { // if the table is the managed table + Path oldPath = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR), + databaseName, simpleTableName); + Path newPath = StorageUtil.concatPath(context.getConf().getVar(TajoConf.ConfVars.WAREHOUSE_DIR), + databaseName, alterTable.getNewTableName()); + FileSystem fs = oldPath.getFileSystem(context.getConf()); + + if (!fs.exists(oldPath)) { + throw new IOException("No such a table directory: " + oldPath); + } + if (fs.exists(newPath)) { + throw new IOException("Already table directory exists: " + newPath); + } + + fs.rename(oldPath, newPath); + } + catalog.alterTable(CatalogUtil.renameTable(qualifiedName, alterTable.getNewTableName(), + AlterTableType.RENAME_TABLE)); + break; + case RENAME_COLUMN: + if (existColumnName(qualifiedName, alterTable.getNewColumnName())) { + throw new ColumnNameAlreadyExistException(alterTable.getNewColumnName()); + } + catalog.alterTable(CatalogUtil.renameColumn(qualifiedName, alterTable.getColumnName(), alterTable.getNewColumnName(), AlterTableType.RENAME_COLUMN)); + break; + case ADD_COLUMN: + if (existColumnName(qualifiedName, alterTable.getAddNewColumn().getSimpleName())) { + throw new ColumnNameAlreadyExistException(alterTable.getAddNewColumn().getSimpleName()); + } + catalog.alterTable(CatalogUtil.addNewColumn(qualifiedName, alterTable.getAddNewColumn(), AlterTableType.ADD_COLUMN)); + break; + default: + //TODO + } + } + + private boolean existColumnName(String tableName, String columnName) { + final TableDesc tableDesc = catalog.getTableDesc(tableName); + return tableDesc.getSchema().containsByName(columnName) ? true : false; + } + private TableDesc createTable(Session session, CreateTableNode createTable, boolean ifNotExists) throws IOException { TableMeta meta; @@ -287,7 +366,18 @@ public class GlobalEngine extends AbstractService { if(createTable.isExternal()){ Preconditions.checkState(createTable.hasPath(), "ERROR: LOCATION must be given."); } else { - Path tablePath = new Path(sm.getWarehouseDir(), createTable.getTableName()); + String databaseName; + String tableName; + if (CatalogUtil.isFQTableName(createTable.getTableName())) { + databaseName = CatalogUtil.extractQualifier(createTable.getTableName()); + tableName = CatalogUtil.extractSimpleName(createTable.getTableName()); + } else { + databaseName = session.getCurrentDatabase(); + tableName = createTable.getTableName(); + } + + // create a table directory (i.e., ${WAREHOUSE_DIR}/${DATABASE_NAME}/${TABLE_NAME} ) + Path tablePath = StorageUtil.concatPath(sm.getWarehouseDir(), databaseName, tableName); createTable.setPath(tablePath); } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java index 65a0d47..784fea5 100644 --- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java +++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/QueryTestCaseBase.java @@ -23,10 +23,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.tajo.algebra.CreateTable; -import org.apache.tajo.algebra.DropTable; -import org.apache.tajo.algebra.Expr; -import org.apache.tajo.algebra.OpType; +import org.apache.tajo.algebra.*; import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.CatalogService; import org.apache.tajo.catalog.CatalogUtil; @@ -321,6 +318,15 @@ public class QueryTestCaseBase { assertTrue(client.existTable(tableName)); } + public void assertColumnExists(String tableName,String columnName) throws ServiceException { + TableDesc tableDesc = fetchTableMetaData(tableName); + assertTrue(tableDesc.getSchema().containsByName(columnName)); + } + + private TableDesc fetchTableMetaData(String tableName) throws ServiceException { + return client.getTableDesc(tableName); + } + public void assertTableNotExists(String tableName) throws ServiceException { assertTrue(!client.existTable(tableName)); } @@ -403,7 +409,7 @@ public class QueryTestCaseBase { } private List executeDDL(String ddlFileName, @Nullable String dataFileName, boolean isLocalTable, - @Nullable String [] args) throws Exception { + @Nullable String[] args) throws Exception { Path ddlFilePath = new Path(currentQueryPath, ddlFileName); FileSystem fs = ddlFilePath.getFileSystem(conf); @@ -431,7 +437,7 @@ public class QueryTestCaseBase { TableDesc createdTable = client.getTableDesc(tableName); String createdTableName = createdTable.getName(); - assertTrue("table '" + createdTableName + "' creation check", client.existTable(createdTableName)); + assertTrue("table '" + createdTableName + "' creation check", client.existTable(createdTableName)); if (isLocalTable) { createdTableGlobalSet.add(createdTableName); createdTableNames.add(tableName); @@ -447,6 +453,14 @@ public class QueryTestCaseBase { if (isLocalTable) { createdTableGlobalSet.remove(tableName); } + } else if (expr.getType() == OpType.AlterTable) { + AlterTable alterTable = (AlterTable) expr; + String tableName = alterTable.getTableName(); + assertTrue("table '" + tableName + "' existence check", client.existTable(tableName)); + client.updateQuery(compiled); + if (isLocalTable) { + createdTableGlobalSet.remove(tableName); + } } else { assertTrue(ddlFilePath + " is not a Create or Drop Table statement", false); } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java new file mode 100644 index 0000000..5779347 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestAlterTable.java @@ -0,0 +1,53 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.query; + +import org.apache.tajo.IntegrationTest; +import org.apache.tajo.QueryTestCaseBase; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.List; + +@Category(IntegrationTest.class) +public class TestAlterTable extends QueryTestCaseBase { + @Test + public final void testAlterTableName() throws Exception { + List createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "ABC"); + assertTableExists(createdNames.get(0)); + executeDDL("alter_table_rename_table_ddl.sql", null); + assertTableExists("DEF"); + + } + + @Test + public final void testAlterTableColumnName() throws Exception { + List createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "XYZ"); + executeDDL("alter_table_rename_column_ddl.sql", null); + assertColumnExists(createdNames.get(0),"renum"); + } + + @Test + public final void testAlterTableAddNewColumn() throws Exception { + List createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "EFG"); + executeDDL("alter_table_add_new_column_ddl.sql", null); + assertColumnExists(createdNames.get(0),"cool"); + } + +} http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java index e6840f6..72e92d2 100644 --- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java +++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCTASQuery.java @@ -212,13 +212,15 @@ public class TestCTASQuery extends QueryTestCaseBase { @Test public final void testCtasWithManagedTable() throws Exception { - ResultSet res = executeFile("CtasWithManagedTable.sql"); - res.close(); - - assertFalse(client.existTable("MANAGED_TABLE1")); - assertTrue(client.existTable("\"MANAGED_TABLE1\"")); - TableDesc desc = client.getTableDesc("\"MANAGED_TABLE1\""); - assertNotNull(desc); - assertEquals("MANAGED_TABLE1", desc.getPath().getName()); + if (!testingCluster.isHCatalogStoreRunning()) { + ResultSet res = executeFile("CtasWithManagedTable.sql"); + res.close(); + + assertFalse(client.existTable("MANAGED_TABLE1")); + assertTrue(client.existTable("\"MANAGED_TABLE1\"")); + TableDesc desc = client.getTableDesc("\"MANAGED_TABLE1\""); + assertNotNull(desc); + assertEquals("MANAGED_TABLE1", desc.getPath().getName()); + } } } http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java index a5b57b2..a6bb467 100644 --- a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java +++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestCreateTable.java @@ -18,14 +18,23 @@ package org.apache.tajo.engine.query; +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.TableDesc; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.storage.StorageUtil; import org.junit.Test; 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; + @Category(IntegrationTest.class) public class TestCreateTable extends QueryTestCaseBase { @@ -79,6 +88,56 @@ public class TestCreateTable extends QueryTestCaseBase { assertDatabaseNotExists("D2"); } + private final void assertPathOfCreatedTable(final String databaseName, + final String originalTableName, + final String newTableName, + String createTableStmt) throws Exception { + // create one table + executeString("CREATE DATABASE " + CatalogUtil.denormalizeIdentifier(databaseName)).close(); + getClient().existDatabase(CatalogUtil.denormalizeIdentifier(databaseName)); + final String oldFQTableName = CatalogUtil.buildFQName(databaseName, originalTableName); + + ResultSet res = executeString(createTableStmt); + res.close(); + assertTableExists(CatalogUtil.denormalizeIdentifier(oldFQTableName)); + TableDesc oldTableDesc = client.getTableDesc(CatalogUtil.denormalizeIdentifier(oldFQTableName)); + + + // checking the existence of the table directory and validating the path + FileSystem fs = testingCluster.getMaster().getStorageManager().getFileSystem(); + Path warehouseDir = TajoConf.getWarehouseDir(testingCluster.getConfiguration()); + assertTrue(fs.exists(oldTableDesc.getPath())); + assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, originalTableName), oldTableDesc.getPath()); + + // Rename + client.executeQuery("ALTER TABLE " + CatalogUtil.denormalizeIdentifier(oldFQTableName) + + " RENAME to " + CatalogUtil.denormalizeIdentifier(newTableName)); + + // checking the existence of the new table directory and validating the path + final String newFQTableName = CatalogUtil.buildFQName(databaseName, newTableName); + TableDesc newTableDesc = client.getTableDesc(CatalogUtil.denormalizeIdentifier(newFQTableName)); + assertTrue(fs.exists(newTableDesc.getPath())); + assertEquals(StorageUtil.concatPath(warehouseDir, databaseName, newTableName), newTableDesc.getPath()); + } + + @Test + public final void testCreatedTableViaCTASAndVerifyPath() throws Exception { + assertPathOfCreatedTable("d4", "old_table", "new_mgmt_table", + "CREATE TABLE d4.old_table AS SELECT * FROM default.lineitem;"); + } + + @Test + public final void testCreatedTableJustCreatedAndVerifyPath() throws Exception { + assertPathOfCreatedTable("d5", "old_table", "new_mgmt_table", "CREATE TABLE d5.old_table (age integer);"); + } + + @Test + public final void testCreatedTableWithQuotedIdentifierAndVerifyPath() throws Exception { + if (!testingCluster.isHCatalogStoreRunning()) { + assertPathOfCreatedTable("D6", "OldTable", "NewMgmtTable", "CREATE TABLE \"D6\".\"OldTable\" (age integer);"); + } + } + @Test public final void testCreateTableIfNotExists() throws Exception { executeString("CREATE DATABASE D3;").close(); http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestDropTable.java ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestDropTable.java b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestDropTable.java new file mode 100644 index 0000000..e362eef --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/java/org/apache/tajo/engine/query/TestDropTable.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.query; + +import org.apache.tajo.IntegrationTest; +import org.apache.tajo.QueryTestCaseBase; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.List; + +@Category(IntegrationTest.class) +public class TestDropTable extends QueryTestCaseBase { + + @Test + public final void testDropManagedTable() throws Exception { + List createdNames = executeDDL("table1_ddl.sql", "table1.tbl", "ABC"); + assertTableExists(createdNames.get(0)); + executeDDL("drop_table_ddl.sql", null); + assertTableNotExists("ABC"); + } + +} http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/dataset/TestAlterTable/table1.tbl ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/dataset/TestAlterTable/table1.tbl b/tajo-core/tajo-core-backend/src/test/resources/dataset/TestAlterTable/table1.tbl new file mode 100644 index 0000000..340fe85 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/dataset/TestAlterTable/table1.tbl @@ -0,0 +1,3 @@ +1|abc|2 +2|def|5 +3|ghi|8 http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/dataset/TestDropTable/table1.tbl ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/dataset/TestDropTable/table1.tbl b/tajo-core/tajo-core-backend/src/test/resources/dataset/TestDropTable/table1.tbl new file mode 100644 index 0000000..340fe85 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/dataset/TestDropTable/table1.tbl @@ -0,0 +1,3 @@ +1|abc|2 +2|def|5 +3|ghi|8 http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_add_new_column_ddl.sql ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_add_new_column_ddl.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_add_new_column_ddl.sql new file mode 100644 index 0000000..567aed8 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_add_new_column_ddl.sql @@ -0,0 +1 @@ +ALTER TABLE EFG ADD COLUMN cool text; http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_column_ddl.sql ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_column_ddl.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_column_ddl.sql new file mode 100644 index 0000000..7c326ff --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_column_ddl.sql @@ -0,0 +1 @@ +ALTER TABLE XYZ RENAME COLUMN num TO renum; http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_table_ddl.sql ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_table_ddl.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_table_ddl.sql new file mode 100644 index 0000000..eeab457 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/alter_table_rename_table_ddl.sql @@ -0,0 +1 @@ + ALTER TABLE ABC RENAME TO DEF; http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/table1_ddl.sql ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/table1_ddl.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/table1_ddl.sql new file mode 100644 index 0000000..8d7fba0 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestAlterTable/table1_ddl.sql @@ -0,0 +1 @@ +CREATE EXTERNAL TABLE ${0} (id int, str text, num int) using csv location ${table.path}; http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/drop_table_ddl.sql ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/drop_table_ddl.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/drop_table_ddl.sql new file mode 100644 index 0000000..c770b43 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/drop_table_ddl.sql @@ -0,0 +1 @@ +DROP TABLE ABC PURGE \ No newline at end of file http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/table1_ddl.sql ---------------------------------------------------------------------- diff --git a/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/table1_ddl.sql b/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/table1_ddl.sql new file mode 100644 index 0000000..0d56ec8 --- /dev/null +++ b/tajo-core/tajo-core-backend/src/test/resources/queries/TestDropTable/table1_ddl.sql @@ -0,0 +1 @@ +CREATE TABLE ${0} (id int, str text, num int); http://git-wip-us.apache.org/repos/asf/tajo/blob/bd418a5c/tajo-docs/src/main/sphinx/functions/datetime_func_and_operators.rst ---------------------------------------------------------------------- diff --git a/tajo-docs/src/main/sphinx/functions/datetime_func_and_operators.rst b/tajo-docs/src/main/sphinx/functions/datetime_func_and_operators.rst index 51156fa..e31caba 100644 --- a/tajo-docs/src/main/sphinx/functions/datetime_func_and_operators.rst +++ b/tajo-docs/src/main/sphinx/functions/datetime_func_and_operators.rst @@ -2,4 +2,41 @@ DateTime Functions and Operators ******************************** -.. todo:: \ No newline at end of file +.. function:: utc_usec_to (string text , long timestamp , int dayOfWeek) + + * If the **first parameter** is 'day'. + + Shifts and return a UNIX timestamp in microseconds to the beginning of the day it occurs in. + For example, if unix_timestamp occurs on May 19th at 08:58, this function returns a UNIX timestamp for May 19th at 00:00 (midnight). + + * If the **first parameter** is 'hour'. + + Shifts and return a UNIX timestamp in microseconds to the beginning of the hour it occurs in. + For example, if unix_timestamp occurs at 08:58, this function returns a UNIX timestamp for 08:00 on the same day. + + * If the **first parameter** is 'month'. + + Shifts and return a UNIX timestamp in microseconds to the beginning of the month it occurs in. + For example, if unix_timestamp occurs on March 19th, this function returns a UNIX timestamp for March 1st of the same year. + + * If the **first parameter** is 'year'. + + Returns a UNIX timestamp in microseconds that represents the year of the unix_timestamp argument. + For example, if unix_timestamp occurs in 2010, the function returns 1274259481071200, the microsecond representation of 2010-01-01 00:00. + + * If the **first parameter** is 'week' and **third parameter** is 2 i.e (TUESDAY) + + Returns a UNIX timestamp in microseconds that represents a day in the week of the + For example, if unix_timestamp occurs on Friday, 2008-04-11, and you set day_of_week to 2 (Tuesday), the function returns a UNIX timestamp for Tuesday, 2008-04-08. + + :param string: could be 'day' 'hour' 'month' 'year' 'week' + :param long: unix timestamp in microseconds + :param int: day of the week from 0 (Sunday) to 6 (Saturday).Optional parameter required only if first parameter is 'week' + :rtype: long + :alias: utc_usec_to + :example: + + .. code-block:: sql + + SELECT utc_usec_to('day', 1274259481071200); + > 1274227200000000 \ No newline at end of file