You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/07/02 03:49:06 UTC
[01/25] hive git commit: HIVE-11029: hadoop.proxyuser.mapr.groups
does not work to restrict the groups that can be impersonated (Na via Xuefu)
Repository: hive
Updated Branches:
refs/heads/llap 4339fd86c -> 255220119
HIVE-11029: hadoop.proxyuser.mapr.groups does not work to restrict the groups that can be impersonated (Na via Xuefu)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/8ed33774
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/8ed33774
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/8ed33774
Branch: refs/heads/llap
Commit: 8ed337749261ad78becb46a16a350ef23d9f305f
Parents: 88482c3
Author: Xuefu Zhang <xz...@Cloudera.com>
Authored: Mon Jun 29 22:10:13 2015 -0700
Committer: Xuefu Zhang <xz...@Cloudera.com>
Committed: Mon Jun 29 22:10:53 2015 -0700
----------------------------------------------------------------------
.../service/cli/session/HiveSessionImplwithUGI.java | 14 +++++---------
1 file changed, 5 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/8ed33774/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
index 2b5f645..cd3c3f9 100644
--- a/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
+++ b/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
@@ -59,15 +59,11 @@ public class HiveSessionImplwithUGI extends HiveSessionImpl {
if (owner == null) {
throw new HiveSQLException("No username provided for impersonation");
}
- if (UserGroupInformation.isSecurityEnabled()) {
- try {
- sessionUgi = UserGroupInformation.createProxyUser(
- owner, UserGroupInformation.getLoginUser());
- } catch (IOException e) {
- throw new HiveSQLException("Couldn't setup proxy user", e);
- }
- } else {
- sessionUgi = UserGroupInformation.createRemoteUser(owner);
+ try {
+ sessionUgi = UserGroupInformation.createProxyUser(
+ owner, UserGroupInformation.getLoginUser());
+ } catch (IOException e) {
+ throw new HiveSQLException("Couldn't setup proxy user", e);
}
}
[12/25] hive git commit: HIVE-11055 HPL/SQL - Implementing Procedural
SQL in Hive (PL/HQL Contribution) (Dmitry Tolpeko via gates)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
new file mode 100644
index 0000000..9ec8959
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Exec.java
@@ -0,0 +1,1950 @@
+/**
+ * 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.hive.hplsql;
+
+import java.math.BigDecimal;
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.InputStream;
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Stack;
+import java.util.Iterator;
+import java.sql.Connection;
+import java.sql.SQLException;
+
+import org.antlr.v4.runtime.ANTLRInputStream;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.Token;
+import org.antlr.v4.runtime.misc.NotNull;
+import org.antlr.v4.runtime.tree.ParseTree;
+import org.apache.commons.io.FileUtils;
+import org.apache.hive.hplsql.functions.*;
+
+/**
+ * HPL/SQL script executor
+ *
+ */
+public class Exec extends HplsqlBaseVisitor<Integer> {
+
+ public static final String VERSION = "HPL/SQL 0.3.11";
+ public static final String SQLCODE = "SQLCODE";
+ public static final String SQLSTATE = "SQLSTATE";
+ public static final String HOSTCODE = "HOSTCODE";
+
+ Exec exec = null;
+ ParseTree tree = null;
+
+ public enum OnError {EXCEPTION, SETERROR, STOP};
+
+ // Scopes of execution (code blocks) with own local variables, parameters and exception handlers
+ Stack<Scope> scopes = new Stack<Scope>();
+ Scope currentScope;
+
+ Stack<Var> stack = new Stack<Var>();
+ Stack<String> labels = new Stack<String>();
+
+ Stack<Signal> signals = new Stack<Signal>();
+ Signal currentSignal;
+ Scope currentHandlerScope;
+ boolean resignal = false;
+
+ HashMap<String, String> managedTables = new HashMap<String, String>();
+ HashMap<String, String> objectMap = new HashMap<String, String>();
+ HashMap<String, String> objectConnMap = new HashMap<String, String>();
+
+ public ArrayList<String> stmtConnList = new ArrayList<String>();
+
+ Arguments arguments = new Arguments();
+ public Conf conf;
+ Expression expr;
+ Function function;
+ Converter converter;
+ Select select;
+ Stmt stmt;
+ Conn conn;
+
+ int rowCount = 0;
+
+ String execString;
+ String execFile;
+ String execMain;
+ StringBuilder localUdf = new StringBuilder();
+ boolean initRoutines = false;
+ public boolean buildSql = false;
+ boolean udfRegistered = false;
+ boolean udfRun = false;
+
+ boolean dotHplsqlrcExists = false;
+ boolean hplsqlrcExists = false;
+
+ boolean trace = false;
+ boolean info = true;
+ boolean offline = false;
+
+ Exec() {
+ exec = this;
+ }
+
+ Exec(Exec exec) {
+ this.exec = exec;
+ }
+
+ /**
+ * Set a variable using a value from the parameter or the stack
+ */
+ public Var setVariable(String name, Var value) {
+ if (value == null || value == Var.Empty) {
+ if (exec.stack.empty()) {
+ return Var.Empty;
+ }
+ value = exec.stack.pop();
+ }
+ if (name.startsWith("hplsql.")) {
+ exec.conf.setOption(name, value.toString());
+ return Var.Empty;
+ }
+ Var var = findVariable(name);
+ if (var != null) {
+ var.cast(value);
+ }
+ else {
+ var = new Var(value);
+ var.setName(name);
+ exec.currentScope.addVariable(var);
+ }
+ return var;
+ }
+
+ public Var setVariable(String name) {
+ return setVariable(name, Var.Empty);
+ }
+
+ public Var setVariable(String name, String value) {
+ return setVariable(name, new Var(value));
+ }
+
+ public Var setVariable(String name, int value) {
+ return setVariable(name, new Var(new Long(value)));
+ }
+
+ /**
+ * Set variable to NULL
+ */
+ public Var setVariableToNull(String name) {
+ Var var = findVariable(name);
+ if (var != null) {
+ var.removeValue();
+ }
+ else {
+ var = new Var();
+ var.setName(name);
+ exec.currentScope.addVariable(var);
+ }
+ return var;
+ }
+
+ /**
+ * Add a local variable to the current scope
+ */
+ public void addVariable(Var var) {
+ if (exec.currentScope != null) {
+ exec.currentScope.addVariable(var);
+ }
+ }
+
+ /**
+ * Add a condition handler to the current scope
+ */
+ public void addHandler(Handler handler) {
+ if (exec.currentScope != null) {
+ exec.currentScope.addHandler(handler);
+ }
+ }
+
+ /**
+ * Push a value to the stack
+ */
+ public void stackPush(Var var) {
+ exec.stack.push(var);
+ }
+
+ /**
+ * Push a string value to the stack
+ */
+ public void stackPush(String val) {
+ exec.stack.push(new Var(val));
+ }
+
+ public void stackPush(StringBuilder val) {
+ stackPush(val.toString());
+ }
+
+ /**
+ * Push a boolean value to the stack
+ */
+ public void stackPush(boolean val) {
+ exec.stack.push(new Var(val));
+ }
+
+ /**
+ * Select a value from the stack, but not remove
+ */
+ public Var stackPeek() {
+ return exec.stack.peek();
+ }
+
+ /**
+ * Pop a value from the stack
+ */
+ public Var stackPop() {
+ if (!exec.stack.isEmpty()) {
+ return exec.stack.pop();
+ }
+ return null;
+ }
+
+ /**
+ * Find an existing variable by name
+ */
+ public Var findVariable(String name) {
+ Scope cur = exec.currentScope;
+ String name2 = null;
+ if (name.startsWith(":")) {
+ name2 = name.substring(1);
+ }
+ while (cur != null) {
+ for (Var v : cur.vars) {
+ if (name.equalsIgnoreCase(v.getName()) ||
+ (name2 != null && name2.equalsIgnoreCase(v.getName()))) {
+ return v;
+ }
+ }
+ cur = cur.parent;
+ }
+ return null;
+ }
+
+ public Var findVariable(Var name) {
+ return findVariable(name.getName());
+ }
+
+ /**
+ * Enter a new scope
+ */
+ public void enterScope(Scope.Type type) {
+ exec.currentScope = new Scope(exec.currentScope, type);
+ exec.scopes.push(exec.currentScope);
+ }
+
+ /**
+ * Leave the current scope
+ */
+ public void leaveScope() {
+ if (!exec.signals.empty()) {
+ Scope scope = exec.scopes.peek();
+ Signal signal = exec.signals.peek();
+ if (exec.conf.onError != OnError.SETERROR) {
+ runExitHandler();
+ }
+ if (signal.type == Signal.Type.LEAVE_ROUTINE && scope.type == Scope.Type.ROUTINE) {
+ exec.signals.pop();
+ }
+ }
+ exec.currentScope = exec.scopes.pop().getParent();
+ }
+
+ /**
+ * Send a signal
+ */
+ public void signal(Signal signal) {
+ exec.signals.push(signal);
+ }
+
+ public void signal(Signal.Type type, String value, Exception exception) {
+ signal(new Signal(type, value, exception));
+ }
+
+ public void signal(Signal.Type type, String value) {
+ signal(type, value, null);
+ }
+
+ public void signal(Signal.Type type) {
+ signal(type, null, null);
+ }
+
+ public void signal(Query query) {
+ setSqlCode(query.getException());
+ signal(Signal.Type.SQLEXCEPTION, query.errorText(), query.getException());
+ }
+
+ public void signal(Exception exception) {
+ setSqlCode(exception);
+ signal(Signal.Type.SQLEXCEPTION, exception.getMessage(), exception);
+ }
+
+ /**
+ * Resignal the condition
+ */
+ public void resignal() {
+ resignal(exec.currentSignal);
+ }
+
+ public void resignal(Signal signal) {
+ if (signal != null) {
+ exec.resignal = true;
+ signal(signal);
+ }
+ }
+
+ /**
+ * Run CONTINUE handlers
+ */
+ boolean runContinueHandler() {
+ Scope cur = exec.currentScope;
+ exec.currentSignal = exec.signals.pop();
+ while (cur != null) {
+ for (Handler h : cur.handlers) {
+ if (h.execType != Handler.ExecType.CONTINUE) {
+ continue;
+ }
+ if ((h.type != Signal.Type.USERDEFINED && h.type == exec.currentSignal.type) ||
+ (h.type == Signal.Type.USERDEFINED && h.type == exec.currentSignal.type &&
+ h.value.equalsIgnoreCase(exec.currentSignal.value))) {
+ trace(h.ctx, "CONTINUE HANDLER");
+ enterScope(Scope.Type.HANDLER);
+ exec.currentHandlerScope = h.scope;
+ visit(h.ctx.single_block_stmt());
+ leaveScope();
+ exec.currentSignal = null;
+ return true;
+ }
+ }
+ cur = cur.parent;
+ }
+ exec.signals.push(exec.currentSignal);
+ exec.currentSignal = null;
+ return false;
+ }
+
+ /**
+ * Run EXIT handler defined for the current scope
+ */
+ boolean runExitHandler() {
+ exec.currentSignal = exec.signals.pop();
+ for (Handler h : currentScope.handlers) {
+ if (h.execType != Handler.ExecType.EXIT) {
+ continue;
+ }
+ if ((h.type != Signal.Type.USERDEFINED && h.type == exec.currentSignal.type) ||
+ (h.type == Signal.Type.USERDEFINED && h.type == exec.currentSignal.type &&
+ h.value.equalsIgnoreCase(currentSignal.value))) {
+ trace(h.ctx, "EXIT HANDLER");
+ enterScope(Scope.Type.HANDLER);
+ exec.currentHandlerScope = h.scope;
+ visit(h.ctx.single_block_stmt());
+ leaveScope();
+ exec.currentSignal = null;
+ return true;
+ }
+ }
+ exec.signals.push(exec.currentSignal);
+ exec.currentSignal = null;
+ return false;
+ }
+
+ /**
+ * Pop the last signal
+ */
+ public Signal signalPop() {
+ if (!exec.signals.empty()) {
+ return exec.signals.pop();
+ }
+ return null;
+ }
+
+ /**
+ * Peek the last signal
+ */
+ public Signal signalPeek() {
+ if (!exec.signals.empty()) {
+ return exec.signals.peek();
+ }
+ return null;
+ }
+
+ /**
+ * Pop the current label
+ */
+ public String labelPop() {
+ if(!exec.labels.empty()) {
+ return exec.labels.pop();
+ }
+ return "";
+ }
+
+ /**
+ * Execute a SQL query (SELECT)
+ */
+ public Query executeQuery(ParserRuleContext ctx, Query query, String connProfile) {
+ if (!exec.offline) {
+ exec.rowCount = 0;
+ exec.conn.executeQuery(query, connProfile);
+ return query;
+ }
+ setSqlNoData();
+ trace(ctx, "Not executed - offline mode set");
+ return query;
+ }
+
+ public Query executeQuery(ParserRuleContext ctx, String sql, String connProfile) {
+ return executeQuery(ctx, new Query(sql), connProfile);
+ }
+
+ /**
+ * Execute a SQL statement
+ */
+ public Query executeSql(ParserRuleContext ctx, String sql, String connProfile) {
+ if (!exec.offline) {
+ exec.rowCount = 0;
+ Query query = conn.executeSql(sql, connProfile);
+ exec.rowCount = query.getRowCount();
+ return query;
+ }
+ trace(ctx, "Not executed - offline mode set");
+ return new Query("");
+ }
+
+ /**
+ * Close the query object
+ */
+ public void closeQuery(Query query, String conn) {
+ if(!exec.offline) {
+ exec.conn.closeQuery(query, conn);
+ }
+ }
+
+ /**
+ * Register JARs, FILEs and CREATE TEMPORARY FUNCTION for UDF call
+ */
+ public void registerUdf() {
+ if (udfRegistered) {
+ return;
+ }
+ ArrayList<String> sql = new ArrayList<String>();
+ String dir = Utils.getExecDir();
+ sql.add("ADD JAR " + dir + "hplsql.jar");
+ sql.add("ADD JAR " + dir + "antlr-runtime-4.5.jar");
+ sql.add("ADD FILE " + dir + Conf.SITE_XML);
+ if (dotHplsqlrcExists) {
+ sql.add("ADD FILE " + dir + Conf.DOT_HPLSQLRC);
+ }
+ if (hplsqlrcExists) {
+ sql.add("ADD FILE " + dir + Conf.HPLSQLRC);
+ }
+ String lu = createLocalUdf();
+ if (lu != null) {
+ sql.add("ADD FILE " + lu);
+ }
+ sql.add("CREATE TEMPORARY FUNCTION hplsql AS 'org.apache.hive.hplsql.Udf'");
+ exec.conn.addPreSql(exec.conf.defaultConnection, sql);
+ udfRegistered = true;
+ }
+
+ /**
+ * Initialize options
+ */
+ void initOptions() {
+ Iterator<Map.Entry<String,String>> i = exec.conf.iterator();
+ while (i.hasNext()) {
+ Entry<String,String> item = (Entry<String,String>)i.next();
+ String key = (String)item.getKey();
+ String value = (String)item.getValue();
+ if (key == null || value == null) {
+ continue;
+ }
+ else if (key.compareToIgnoreCase(Conf.CONN_DEFAULT) == 0) {
+ exec.conf.defaultConnection = value;
+ }
+ else if (key.startsWith("hplsql.conn.init.")) {
+ exec.conn.addConnectionInit(key.substring(16), value);
+ }
+ else if (key.startsWith(Conf.CONN_CONVERT)) {
+ exec.conf.setConnectionConvert(key.substring(19), value);
+ }
+ else if (key.startsWith("hplsql.conn.")) {
+ exec.conn.addConnection(key.substring(11), value);
+ }
+ else if (key.startsWith("hplsql.")) {
+ exec.conf.setOption(key, value);
+ }
+ }
+ }
+
+ /**
+ * Set SQLCODE
+ */
+ public void setSqlCode(int sqlcode) {
+ Var var = findVariable(SQLCODE);
+ if (var != null) {
+ var.setValue(new Long(sqlcode));
+ }
+ }
+
+ public void setSqlCode(Exception exception) {
+ if (exception instanceof SQLException) {
+ setSqlCode(((SQLException)exception).getErrorCode());
+ setSqlState(((SQLException)exception).getSQLState());
+ }
+ else {
+ setSqlCode(-1);
+ setSqlState("02000");
+ }
+ }
+
+ /**
+ * Set SQLSTATE
+ */
+ public void setSqlState(String sqlstate) {
+ Var var = findVariable(SQLSTATE);
+ if (var != null) {
+ var.setValue(sqlstate);
+ }
+ }
+
+ /**
+ * Set HOSTCODE
+ */
+ public void setHostCode(int code) {
+ Var var = findVariable(HOSTCODE);
+ if (var != null) {
+ var.setValue(new Long(code));
+ }
+ }
+
+ /**
+ * Set successful execution for SQL
+ */
+ public void setSqlSuccess() {
+ setSqlCode(0);
+ setSqlState("00000");
+ }
+
+ /**
+ * Set SQL_NO_DATA as the result of SQL execution
+ */
+ public void setSqlNoData() {
+ setSqlCode(100);
+ setSqlState("01000");
+ }
+
+ /**
+ * Compile and run PL/HQL script
+ */
+ public Integer run(String[] args) throws Exception {
+ if (init(args) != 0) {
+ return 1;
+ }
+ Var result = run();
+ if (result != null) {
+ System.out.println(result.toString());
+ }
+ cleanup();
+ printExceptions();
+ return getProgramReturnCode();
+ }
+
+ /**
+ * Run already compiled PL/HQL script (also used from Hive UDF)
+ */
+ public Var run() {
+ if (tree == null) {
+ return null;
+ }
+ if (execMain != null) {
+ initRoutines = true;
+ visit(tree);
+ initRoutines = false;
+ exec.function.execProc(execMain);
+ }
+ else {
+ visit(tree);
+ }
+ return stackPop();
+ }
+
+ /**
+ * Initialize PL/HQL
+ */
+ Integer init(String[] args) throws Exception {
+ if (!parseArguments(args)) {
+ return 1;
+ }
+ conf = new Conf();
+ conf.init();
+ conn = new Conn(this);
+ initOptions();
+
+ expr = new Expression(this);
+ select = new Select(this);
+ stmt = new Stmt(this);
+ converter = new Converter(this);
+
+ function = new Function(this);
+ new FunctionDatetime(this).register(function);
+ new FunctionMisc(this).register(function);
+ new FunctionString(this).register(function);
+ new FunctionOra(this).register(function);
+
+ enterScope(Scope.Type.FILE);
+ addVariable(new Var(SQLCODE, Var.Type.BIGINT, 0L));
+ addVariable(new Var(SQLSTATE, Var.Type.STRING, "00000"));
+ addVariable(new Var(HOSTCODE, Var.Type.BIGINT, 0L));
+
+ for (Map.Entry<String, String> v : arguments.getVars().entrySet()) {
+ addVariable(new Var(v.getKey(), Var.Type.STRING, v.getValue()));
+ }
+ InputStream input = null;
+ if (execString != null) {
+ input = new ByteArrayInputStream(execString.getBytes("UTF-8"));
+ }
+ else {
+ input = new FileInputStream(execFile);
+ }
+ HplsqlLexer lexer = new HplsqlLexer(new ANTLRInputStream(input));
+ CommonTokenStream tokens = new CommonTokenStream(lexer);
+ HplsqlParser parser = new HplsqlParser(tokens);
+ tree = parser.program();
+ if (trace) {
+ System.err.println("Configuration file: " + conf.getLocation());
+ System.err.println("Parser tree: " + tree.toStringTree(parser));
+ }
+ includeRcFile();
+ return 0;
+ }
+
+ /**
+ * Parse command line arguments
+ */
+ boolean parseArguments(String[] args) {
+ boolean parsed = arguments.parse(args);
+ if (parsed && arguments.hasVersionOption()) {
+ System.err.println(VERSION);
+ return false;
+ }
+ if (!parsed || arguments.hasHelpOption() ||
+ (arguments.getExecString() == null && arguments.getFileName() == null)) {
+ arguments.printHelp();
+ return false;
+ }
+ execString = arguments.getExecString();
+ execFile = arguments.getFileName();
+ execMain = arguments.getMain();
+ if (arguments.hasTraceOption()) {
+ trace = true;
+ }
+ if (arguments.hasOfflineOption()) {
+ offline = true;
+ }
+ if (execString != null && execFile != null) {
+ System.err.println("The '-e' and '-f' options cannot be specified simultaneously.");
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * Include statements from .hplsqlrc and hplsql rc files
+ */
+ void includeRcFile() {
+ if (includeFile(Conf.DOT_HPLSQLRC)) {
+ dotHplsqlrcExists = true;
+ }
+ else {
+ if (includeFile(Conf.HPLSQLRC)) {
+ hplsqlrcExists = true;
+ }
+ }
+ if (udfRun) {
+ includeFile(Conf.HPLSQL_LOCALS_SQL);
+ }
+ }
+
+ /**
+ * Include statements from a file
+ */
+ boolean includeFile(String file) {
+ try {
+ String content = FileUtils.readFileToString(new java.io.File(file), "UTF-8");
+ if (content != null && !content.isEmpty()) {
+ if (trace) {
+ trace(null, "INLCUDE CONTENT " + file + " (non-empty)");
+ }
+ new Exec(this).include(content);
+ return true;
+ }
+ }
+ catch (Exception e) {}
+ return false;
+ }
+
+ /**
+ * Execute statements from an include file
+ */
+ void include(String content) throws Exception {
+ InputStream input = new ByteArrayInputStream(content.getBytes("UTF-8"));
+ HplsqlLexer lexer = new HplsqlLexer(new ANTLRInputStream(input));
+ CommonTokenStream tokens = new CommonTokenStream(lexer);
+ HplsqlParser parser = new HplsqlParser(tokens);
+ ParseTree tree = parser.program();
+ visit(tree);
+ }
+
+ /**
+ * Start executing PL/HQL script
+ */
+ @Override
+ public Integer visitProgram(HplsqlParser.ProgramContext ctx) {
+ enterScope(Scope.Type.FILE);
+ Integer rc = visitChildren(ctx);
+ leaveScope();
+ return rc;
+ }
+
+ /**
+ * Enter BEGIN-END block
+ */
+ @Override
+ public Integer visitBegin_end_block(HplsqlParser.Begin_end_blockContext ctx) {
+ enterScope(Scope.Type.BEGIN_END);
+ Integer rc = visitChildren(ctx);
+ leaveScope();
+ return rc;
+ }
+
+ /**
+ * Free resources before exit
+ */
+ void cleanup() {
+ for (Map.Entry<String, String> i : managedTables.entrySet()) {
+ String sql = "DROP TABLE IF EXISTS " + i.getValue();
+ Query query = executeSql(null, sql, exec.conf.defaultConnection);
+ closeQuery(query, exec.conf.defaultConnection);
+ if (trace) {
+ trace(null, sql);
+ }
+ }
+ }
+
+ /**
+ * Output information about unhandled exceptions
+ */
+ void printExceptions() {
+ while (!signals.empty()) {
+ Signal sig = signals.pop();
+ if (sig.type == Signal.Type.SQLEXCEPTION) {
+ System.err.println("Unhandled exception in PL/HQL");
+ }
+ if (sig.exception != null) {
+ sig.exception.printStackTrace();
+ }
+ else if (sig.value != null) {
+ System.err.println(sig.value);
+ }
+ }
+ }
+
+ /**
+ * Get the program return code
+ */
+ Integer getProgramReturnCode() {
+ Integer rc = 0;
+ if(!signals.empty()) {
+ Signal sig = signals.pop();
+ if(sig.type == Signal.Type.LEAVE_ROUTINE && sig.value != null) {
+ try {
+ rc = Integer.parseInt(sig.value);
+ }
+ catch(NumberFormatException e) {
+ rc = 1;
+ }
+ }
+ }
+ return rc;
+ }
+
+ /**
+ * Executing a statement
+ */
+ @Override
+ public Integer visitStmt(HplsqlParser.StmtContext ctx) {
+ if (ctx.semicolon_stmt() != null) {
+ return 0;
+ }
+ if (initRoutines && ctx.create_procedure_stmt() == null && ctx.create_function_stmt() == null) {
+ return 0;
+ }
+ if (exec.resignal) {
+ if (exec.currentScope != exec.currentHandlerScope.parent) {
+ return 0;
+ }
+ exec.resignal = false;
+ }
+ if (!exec.signals.empty() && exec.conf.onError != OnError.SETERROR) {
+ if (!runContinueHandler()) {
+ return 0;
+ }
+ }
+ Var prevResult = stackPop();
+ if (prevResult != null) {
+ System.out.println(prevResult.toString());
+ }
+ return visitChildren(ctx);
+ }
+
+ /**
+ * Executing or building SELECT statement
+ */
+ @Override
+ public Integer visitSelect_stmt(HplsqlParser.Select_stmtContext ctx) {
+ return exec.select.select(ctx);
+ }
+
+ @Override
+ public Integer visitCte_select_stmt(HplsqlParser.Cte_select_stmtContext ctx) {
+ return exec.select.cte(ctx);
+ }
+
+ @Override
+ public Integer visitFullselect_stmt(HplsqlParser.Fullselect_stmtContext ctx) {
+ return exec.select.fullselect(ctx);
+ }
+
+ @Override
+ public Integer visitSubselect_stmt(HplsqlParser.Subselect_stmtContext ctx) {
+ return exec.select.subselect(ctx);
+ }
+
+ @Override
+ public Integer visitSelect_list(HplsqlParser.Select_listContext ctx) {
+ return exec.select.selectList(ctx);
+ }
+
+ @Override
+ public Integer visitFrom_clause(HplsqlParser.From_clauseContext ctx) {
+ return exec.select.from(ctx);
+ }
+
+ @Override
+ public Integer visitFrom_table_name_clause(HplsqlParser.From_table_name_clauseContext ctx) {
+ return exec.select.fromTable(ctx);
+ }
+
+ @Override
+ public Integer visitFrom_join_clause(HplsqlParser.From_join_clauseContext ctx) {
+ return exec.select.fromJoin(ctx);
+ }
+
+ @Override
+ public Integer visitFrom_table_values_clause(HplsqlParser.From_table_values_clauseContext ctx) {
+ return exec.select.fromTableValues(ctx);
+ }
+
+ @Override
+ public Integer visitWhere_clause(HplsqlParser.Where_clauseContext ctx) {
+ return exec.select.where(ctx);
+ }
+
+ @Override
+ public Integer visitSelect_options_item(HplsqlParser.Select_options_itemContext ctx) {
+ return exec.select.option(ctx);
+ }
+
+ /**
+ * Table name
+ */
+ @Override
+ public Integer visitTable_name(HplsqlParser.Table_nameContext ctx) {
+ String name = ctx.getText().toUpperCase();
+ String actualName = exec.managedTables.get(name);
+ String conn = exec.objectConnMap.get(name);
+ if (conn == null) {
+ conn = conf.defaultConnection;
+ }
+ stmtConnList.add(conn);
+ if (actualName != null) {
+ stackPush(actualName);
+ return 0;
+ }
+ actualName = exec.objectMap.get(name);
+ if (actualName != null) {
+ stackPush(actualName);
+ return 0;
+ }
+ stackPush(ctx.getText());
+ return 0;
+ }
+
+ /**
+ * SQL INSERT statement
+ */
+ @Override
+ public Integer visitInsert_stmt(HplsqlParser.Insert_stmtContext ctx) {
+ return exec.stmt.insert(ctx);
+ }
+
+ /**
+ * EXCEPTION block
+ */
+ @Override
+ public Integer visitException_block_item(HplsqlParser.Exception_block_itemContext ctx) {
+ if (exec.signals.empty()) {
+ return 0;
+ }
+ if (exec.conf.onError == OnError.SETERROR || exec.conf.onError == OnError.STOP) {
+ exec.signals.pop();
+ return 0;
+ }
+ if (ctx.L_ID().toString().equalsIgnoreCase("OTHERS")) {
+ trace(ctx, "EXCEPTION HANDLER");
+ exec.signals.pop();
+ enterScope(Scope.Type.HANDLER);
+ visit(ctx.block());
+ leaveScope();
+ }
+ return 0;
+ }
+
+ /**
+ * DECLARE variable statement
+ */
+ @Override
+ public Integer visitDeclare_var_item(HplsqlParser.Declare_var_itemContext ctx) {
+ String type = ctx.dtype().getText();
+ String len = null;
+ String scale = null;
+ Var default_ = null;
+ if (ctx.dtype_len() != null) {
+ len = ctx.dtype_len().L_INT(0).getText();
+ if (ctx.dtype_len().L_INT(1) != null) {
+ scale = ctx.dtype_len().L_INT(1).getText();
+ }
+ }
+ if (ctx.dtype_default() != null) {
+ default_ = evalPop(ctx.dtype_default());
+ }
+ int cnt = ctx.ident().size(); // Number of variables declared with the same data type and default
+ for (int i = 0; i < cnt; i++) {
+ String name = ctx.ident(i).getText();
+ Var var = new Var(name, type, len, scale, default_);
+ addVariable(var);
+ if (trace) {
+ if (default_ != null) {
+ trace(ctx, "DECLARE " + name + " " + type + " = " + var.toSqlString());
+ }
+ else {
+ trace(ctx, "DECLARE " + name + " " + type);
+ }
+ }
+ }
+ return 0;
+ }
+
+ /**
+ * DECLARE cursor statement
+ */
+ @Override
+ public Integer visitDeclare_cursor_item(HplsqlParser.Declare_cursor_itemContext ctx) {
+ return exec.stmt.declareCursor(ctx);
+ }
+
+ /**
+ * DROP statement
+ */
+ @Override
+ public Integer visitDrop_stmt(HplsqlParser.Drop_stmtContext ctx) {
+ return exec.stmt.drop(ctx);
+ }
+
+ /**
+ * OPEN cursor statement
+ */
+ @Override
+ public Integer visitOpen_stmt(HplsqlParser.Open_stmtContext ctx) {
+ return exec.stmt.open(ctx);
+ }
+
+ /**
+ * FETCH cursor statement
+ */
+ @Override
+ public Integer visitFetch_stmt(HplsqlParser.Fetch_stmtContext ctx) {
+ return exec.stmt.fetch(ctx);
+ }
+
+ /**
+ * CLOSE cursor statement
+ */
+ @Override
+ public Integer visitClose_stmt(HplsqlParser.Close_stmtContext ctx) {
+ return exec.stmt.close(ctx);
+ }
+
+ /**
+ * COPY statement
+ */
+ @Override
+ public Integer visitCopy_stmt(HplsqlParser.Copy_stmtContext ctx) {
+ return new Copy(exec).run(ctx);
+ }
+
+ /**
+ * COPY FROM LOCAL statement
+ */
+ @Override
+ public Integer visitCopy_from_local_stmt(HplsqlParser.Copy_from_local_stmtContext ctx) {
+ return new Copy(exec).runFromLocal(ctx);
+ }
+
+ /**
+ * DECLARE HANDLER statement
+ */
+ @Override
+ public Integer visitDeclare_handler_item(HplsqlParser.Declare_handler_itemContext ctx) {
+ trace(ctx, "DECLARE HANDLER");
+ Handler.ExecType execType = Handler.ExecType.EXIT;
+ Signal.Type type = Signal.Type.SQLEXCEPTION;
+ String value = null;
+ if (ctx.T_CONTINUE() != null) {
+ execType = Handler.ExecType.CONTINUE;
+ }
+ if (ctx.ident() != null) {
+ type = Signal.Type.USERDEFINED;
+ value = ctx.ident().getText();
+ }
+ else if (ctx.T_NOT() != null && ctx.T_FOUND() != null) {
+ type = Signal.Type.NOTFOUND;
+ }
+ addHandler(new Handler(execType, type, value, exec.currentScope, ctx));
+ return 0;
+ }
+
+ /**
+ * DECLARE CONDITION
+ */
+ @Override
+ public Integer visitDeclare_condition_item(HplsqlParser.Declare_condition_itemContext ctx) {
+ return 0;
+ }
+
+ /**
+ * DECLARE TEMPORARY TABLE statement
+ */
+ @Override
+ public Integer visitDeclare_temporary_table_item(HplsqlParser.Declare_temporary_table_itemContext ctx) {
+ return exec.stmt.declareTemporaryTable(ctx);
+ }
+
+ /**
+ * CREATE TABLE statement
+ */
+ @Override
+ public Integer visitCreate_table_stmt(HplsqlParser.Create_table_stmtContext ctx) {
+ return exec.stmt.createTable(ctx);
+ }
+
+ @Override
+ public Integer visitCreate_table_options_hive_item(HplsqlParser.Create_table_options_hive_itemContext ctx) {
+ return exec.stmt.createTableHiveOptions(ctx);
+ }
+
+ /**
+ * CREATE LOCAL TEMPORARY | VOLATILE TABLE statement
+ */
+ @Override
+ public Integer visitCreate_local_temp_table_stmt(HplsqlParser.Create_local_temp_table_stmtContext ctx) {
+ return exec.stmt.createLocalTemporaryTable(ctx);
+ }
+
+ /**
+ * CREATE FUNCTION statement
+ */
+ @Override
+ public Integer visitCreate_function_stmt(HplsqlParser.Create_function_stmtContext ctx) {
+ exec.function.addUserFunction(ctx);
+ addLocalUdf(ctx);
+ return 0;
+ }
+
+ /**
+ * CREATE PROCEDURE statement
+ */
+ @Override
+ public Integer visitCreate_procedure_stmt(HplsqlParser.Create_procedure_stmtContext ctx) {
+ exec.function.addUserProcedure(ctx);
+ addLocalUdf(ctx); // Add procedures as they can be invoked by functions
+ return 0;
+ }
+
+ /**
+ * CREATE INDEX statement
+ */
+ @Override
+ public Integer visitCreate_index_stmt(HplsqlParser.Create_index_stmtContext ctx) {
+ return 0;
+ }
+
+ /**
+ * Add functions and procedures defined in the current script
+ */
+ void addLocalUdf(ParserRuleContext ctx) {
+ if (exec == this) {
+ localUdf.append(exec.getFormattedText(ctx));
+ localUdf.append("\n");
+ }
+ }
+
+ /**
+ * Save local functions and procedures to a file (will be added to the distributed cache)
+ */
+ String createLocalUdf() {
+ if(localUdf.length() == 0) {
+ return null;
+ }
+ try {
+ String file = System.getProperty("user.dir") + "/" + Conf.HPLSQL_LOCALS_SQL;
+ PrintWriter writer = new PrintWriter(file, "UTF-8");
+ writer.print(localUdf);
+ writer.close();
+ return file;
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ }
+ return null;
+ }
+
+ /**
+ * Assignment statement for single value
+ */
+ @Override
+ public Integer visitAssignment_stmt_single_item(HplsqlParser.Assignment_stmt_single_itemContext ctx) {
+ String name = ctx.ident().getText();
+ visit(ctx.expr());
+ Var var = setVariable(name);
+ if (trace) {
+ trace(ctx, "SET " + name + " = " + var.toSqlString());
+ }
+ return 0;
+ }
+
+ /**
+ * Assignment statement for multiple values
+ */
+ @Override
+ public Integer visitAssignment_stmt_multiple_item(HplsqlParser.Assignment_stmt_multiple_itemContext ctx) {
+ int cnt = ctx.ident().size();
+ int ecnt = ctx.expr().size();
+ for (int i = 0; i < cnt; i++) {
+ String name = ctx.ident(i).getText();
+ if (i < ecnt) {
+ visit(ctx.expr(i));
+ Var var = setVariable(name);
+ if (trace) {
+ trace(ctx, "SET " + name + " = " + var.toString());
+ }
+ }
+ }
+ return 0;
+ }
+
+ /**
+ * Assignment from SELECT statement
+ */
+ @Override
+ public Integer visitAssignment_stmt_select_item(HplsqlParser.Assignment_stmt_select_itemContext ctx) {
+ return stmt.assignFromSelect(ctx);
+ }
+
+ /**
+ * Evaluate an expression
+ */
+ @Override
+ public Integer visitExpr(HplsqlParser.ExprContext ctx) {
+ if (exec.buildSql) {
+ exec.expr.execSql(ctx);
+ }
+ else {
+ exec.expr.exec(ctx);
+ }
+ return 0;
+ }
+
+ /**
+ * Evaluate a boolean expression
+ */
+ @Override
+ public Integer visitBool_expr(HplsqlParser.Bool_exprContext ctx) {
+ if (exec.buildSql) {
+ exec.expr.execBoolSql(ctx);
+ }
+ else {
+ exec.expr.execBool(ctx);
+ }
+ return 0;
+ }
+
+ @Override
+ public Integer visitBool_expr_binary(HplsqlParser.Bool_expr_binaryContext ctx) {
+ if (exec.buildSql) {
+ exec.expr.execBoolBinarySql(ctx);
+ }
+ else {
+ exec.expr.execBoolBinary(ctx);
+ }
+ return 0;
+ }
+
+ @Override
+ public Integer visitBool_expr_unary(HplsqlParser.Bool_expr_unaryContext ctx) {
+ if (exec.buildSql) {
+ exec.expr.execBoolUnarySql(ctx);
+ }
+ else {
+ exec.expr.execBoolUnary(ctx);
+ }
+ return 0;
+ }
+
+ /**
+ * Function call
+ */
+ @Override
+ public Integer visitExpr_func(HplsqlParser.Expr_funcContext ctx) {
+ String name = ctx.ident().getText();
+ if (exec.buildSql) {
+ exec.function.execSql(name, ctx.expr_func_params());
+ }
+ else {
+ exec.function.exec(name, ctx.expr_func_params());
+ }
+ return 0;
+ }
+
+ /**
+ * Aggregate or window function call
+ */
+ @Override
+ public Integer visitExpr_agg_window_func(HplsqlParser.Expr_agg_window_funcContext ctx) {
+ exec.function.execAggWindowSql(ctx);
+ return 0;
+ }
+
+ /**
+ * Function with specific syntax
+ */
+ @Override
+ public Integer visitExpr_spec_func(HplsqlParser.Expr_spec_funcContext ctx) {
+ if (exec.buildSql) {
+ exec.function.specExecSql(ctx);
+ }
+ else {
+ exec.function.specExec(ctx);
+ }
+ return 0;
+ }
+
+ /**
+ * INCLUDE statement
+ */
+ @Override
+ public Integer visitInclude_stmt(@NotNull HplsqlParser.Include_stmtContext ctx) {
+ return exec.stmt.include(ctx);
+ }
+
+ /**
+ * IF statement (PL/SQL syntax)
+ */
+ @Override
+ public Integer visitIf_plsql_stmt(HplsqlParser.If_plsql_stmtContext ctx) {
+ return exec.stmt.ifPlsql(ctx);
+ }
+
+ /**
+ * IF statement (Transact-SQL syntax)
+ */
+ @Override
+ public Integer visitIf_tsql_stmt(HplsqlParser.If_tsql_stmtContext ctx) {
+ return exec.stmt.ifTsql(ctx);
+ }
+
+ /**
+ * USE statement
+ */
+ @Override
+ public Integer visitUse_stmt(HplsqlParser.Use_stmtContext ctx) {
+ return exec.stmt.use(ctx);
+ }
+
+ /**
+ * VALUES statement
+ */
+ @Override
+ public Integer visitValues_into_stmt(HplsqlParser.Values_into_stmtContext ctx) {
+ return exec.stmt.values(ctx);
+ }
+
+ /**
+ * WHILE statement
+ */
+ @Override
+ public Integer visitWhile_stmt(HplsqlParser.While_stmtContext ctx) {
+ return exec.stmt.while_(ctx);
+ }
+
+ /**
+ * FOR cursor statement
+ */
+ @Override
+ public Integer visitFor_cursor_stmt(HplsqlParser.For_cursor_stmtContext ctx) {
+ return exec.stmt.forCursor(ctx);
+ }
+
+ /**
+ * FOR (integer range) statement
+ */
+ @Override
+ public Integer visitFor_range_stmt(HplsqlParser.For_range_stmtContext ctx) {
+ return exec.stmt.forRange(ctx);
+ }
+
+ /**
+ * EXEC, EXECUTE and EXECUTE IMMEDIATE statement to execute dynamic SQL
+ */
+ @Override
+ public Integer visitExec_stmt(HplsqlParser.Exec_stmtContext ctx) {
+ return exec.stmt.exec(ctx);
+ }
+
+ /**
+ * CALL statement
+ */
+ @Override
+ public Integer visitCall_stmt(HplsqlParser.Call_stmtContext ctx) {
+ if (exec.function.execProc(ctx.expr_func_params(), ctx.ident().getText())) {
+ return 0;
+ }
+ return -1;
+ }
+
+ /**
+ * EXIT statement (leave the specified loop with a condition)
+ */
+ @Override
+ public Integer visitExit_stmt(HplsqlParser.Exit_stmtContext ctx) {
+ return exec.stmt.exit(ctx);
+ }
+
+ /**
+ * BREAK statement (leave the innermost loop unconditionally)
+ */
+ @Override
+ public Integer visitBreak_stmt(HplsqlParser.Break_stmtContext ctx) {
+ return exec.stmt.break_(ctx);
+ }
+
+ /**
+ * LEAVE statement (leave the specified loop unconditionally)
+ */
+ @Override
+ public Integer visitLeave_stmt(HplsqlParser.Leave_stmtContext ctx) {
+ return exec.stmt.leave(ctx);
+ }
+
+ /**
+ * PRINT statement
+ */
+ @Override
+ public Integer visitPrint_stmt(HplsqlParser.Print_stmtContext ctx) {
+ return exec.stmt.print(ctx);
+ }
+
+ /**
+ * SIGNAL statement
+ */
+ @Override
+ public Integer visitSignal_stmt(HplsqlParser.Signal_stmtContext ctx) {
+ return exec.stmt.signal(ctx);
+ }
+
+ /**
+ * RESIGNAL statement
+ */
+ @Override
+ public Integer visitResignal_stmt(HplsqlParser.Resignal_stmtContext ctx) {
+ return exec.stmt.resignal(ctx);
+ }
+
+ /**
+ * RETURN statement
+ */
+ @Override
+ public Integer visitReturn_stmt(HplsqlParser.Return_stmtContext ctx) {
+ return exec.stmt.return_(ctx);
+ }
+
+ /**
+ * MAP OBJECT statement
+ */
+ @Override
+ public Integer visitMap_object_stmt(HplsqlParser.Map_object_stmtContext ctx) {
+ String source = evalPop(ctx.expr(0)).toString();
+ String target = null;
+ String conn = null;
+ if (ctx.T_TO() != null) {
+ target = evalPop(ctx.expr(1)).toString();
+ exec.objectMap.put(source.toUpperCase(), target);
+ }
+ if (ctx.T_AT() != null) {
+ if (ctx.T_TO() == null) {
+ conn = evalPop(ctx.expr(1)).toString();
+ }
+ else {
+ conn = evalPop(ctx.expr(2)).toString();
+ }
+ exec.objectConnMap.put(source.toUpperCase(), conn);
+ }
+ if (trace) {
+ String log = "MAP OBJECT " + source;
+ if (target != null) {
+ log += " AS " + target;
+ }
+ if (conn != null) {
+ log += " AT " + conn;
+ }
+ trace(ctx, log);
+ }
+ return 0;
+ }
+
+ /**
+ * UPDATE statement
+ */
+ @Override
+ public Integer visitUpdate_stmt(HplsqlParser.Update_stmtContext ctx) {
+ return stmt.update(ctx);
+ }
+
+ /**
+ * DELETE statement
+ */
+ @Override
+ public Integer visitDelete_stmt(HplsqlParser.Delete_stmtContext ctx) {
+ return stmt.delete(ctx);
+ }
+
+ /**
+ * MERGE statement
+ */
+ @Override
+ public Integer visitMerge_stmt(HplsqlParser.Merge_stmtContext ctx) {
+ return stmt.merge(ctx);
+ }
+
+ /**
+ * Run a Hive command line
+ */
+ @Override
+ public Integer visitHive(@NotNull HplsqlParser.HiveContext ctx) {
+ trace(ctx, "HIVE");
+ ArrayList<String> cmd = new ArrayList<String>();
+ cmd.add("hive");
+ Var params = new Var(Var.Type.STRINGLIST, cmd);
+ stackPush(params);
+ visitChildren(ctx);
+ stackPop();
+ try {
+ String[] cmdarr = new String[cmd.size()];
+ cmd.toArray(cmdarr);
+ if(trace) {
+ trace(ctx, "HIVE Parameters: " + Utils.toString(cmdarr, ' '));
+ }
+ if (!offline) {
+ Process p = Runtime.getRuntime().exec(cmdarr);
+ new StreamGobbler(p.getInputStream()).start();
+ new StreamGobbler(p.getErrorStream()).start();
+ int rc = p.waitFor();
+ if (trace) {
+ trace(ctx, "HIVE Process exit code: " + rc);
+ }
+ }
+ } catch (Exception e) {
+ setSqlCode(-1);
+ signal(Signal.Type.SQLEXCEPTION, e.getMessage(), e);
+ return -1;
+ }
+ return 0;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public Integer visitHive_item(HplsqlParser.Hive_itemContext ctx) {
+ Var params = stackPeek();
+ ArrayList<String> a = (ArrayList<String>)params.value;
+ if(ctx.P_e() != null) {
+ a.add("-e");
+ a.add(evalPop(ctx.expr()).toString());
+ }
+ else if(ctx.P_f() != null) {
+ a.add("-f");
+ a.add(evalPop(ctx.expr()).toString());
+ }
+ else if(ctx.P_hiveconf() != null) {
+ a.add("-hiveconf");
+ a.add(ctx.L_ID().toString() + "=" + evalPop(ctx.expr()).toString());
+ }
+ return 0;
+ }
+
+ /**
+ * Executing OS command
+ */
+ @Override
+ public Integer visitHost_cmd(HplsqlParser.Host_cmdContext ctx) {
+ trace(ctx, "HOST");
+ execHost(ctx, ctx.start.getInputStream().getText(
+ new org.antlr.v4.runtime.misc.Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex())));
+ return 0;
+ }
+
+ @Override
+ public Integer visitHost_stmt(HplsqlParser.Host_stmtContext ctx) {
+ trace(ctx, "HOST");
+ execHost(ctx, evalPop(ctx.expr()).toString());
+ return 0;
+ }
+
+ public void execHost(ParserRuleContext ctx, String cmd) {
+ try {
+ if (trace) {
+ trace(ctx, "HOST Command: " + cmd);
+ }
+ Process p = Runtime.getRuntime().exec(cmd);
+ new StreamGobbler(p.getInputStream()).start();
+ new StreamGobbler(p.getErrorStream()).start();
+ int rc = p.waitFor();
+ if (trace) {
+ trace(ctx, "HOST Process exit code: " + rc);
+ }
+ setHostCode(rc);
+ } catch (Exception e) {
+ setHostCode(1);
+ signal(Signal.Type.SQLEXCEPTION);
+ }
+ }
+
+ /**
+ * Standalone expression (as a statement)
+ */
+ @Override
+ public Integer visitExpr_stmt(HplsqlParser.Expr_stmtContext ctx) {
+ visitChildren(ctx);
+ return 0;
+ }
+
+ /**
+ * String concatenation operator
+ */
+ @Override
+ public Integer visitExpr_concat(HplsqlParser.Expr_concatContext ctx) {
+ if (exec.buildSql) {
+ exec.expr.operatorConcatSql(ctx);
+ }
+ else {
+ exec.expr.operatorConcat(ctx);
+ }
+ return 0;
+ }
+
+ /**
+ * Simple CASE expression
+ */
+ @Override
+ public Integer visitExpr_case_simple(HplsqlParser.Expr_case_simpleContext ctx) {
+ if (exec.buildSql) {
+ exec.expr.execSimpleCaseSql(ctx);
+ }
+ else {
+ exec.expr.execSimpleCase(ctx);
+ }
+ return 0;
+ }
+
+ /**
+ * Searched CASE expression
+ */
+ @Override
+ public Integer visitExpr_case_searched(HplsqlParser.Expr_case_searchedContext ctx) {
+ if (exec.buildSql) {
+ exec.expr.execSearchedCaseSql(ctx);
+ }
+ else {
+ exec.expr.execSearchedCase(ctx);
+ }
+ return 0;
+ }
+
+ /**
+ * GET DIAGNOSTICS EXCEPTION statement
+ */
+ @Override
+ public Integer visitGet_diag_stmt_exception_item(HplsqlParser.Get_diag_stmt_exception_itemContext ctx) {
+ return exec.stmt.getDiagnosticsException(ctx);
+ }
+
+ /**
+ * GET DIAGNOSTICS ROW_COUNT statement
+ */
+ @Override
+ public Integer visitGet_diag_stmt_rowcount_item(HplsqlParser.Get_diag_stmt_rowcount_itemContext ctx) {
+ return exec.stmt.getDiagnosticsRowCount(ctx);
+ }
+
+ /**
+ * GRANT statement
+ */
+ @Override
+ public Integer visitGrant_stmt(HplsqlParser.Grant_stmtContext ctx) {
+ trace(ctx, "GRANT");
+ return 0;
+ }
+
+ /**
+ * Label
+ */
+ @Override
+ public Integer visitLabel(HplsqlParser.LabelContext ctx) {
+ exec.labels.push(ctx.L_ID().toString());
+ return 0;
+ }
+
+ /**
+ * Identifier
+ */
+ @Override
+ public Integer visitIdent(HplsqlParser.IdentContext ctx) {
+ String ident = ctx.getText();
+ Var var = findVariable(ident);
+ if (var != null) {
+ if (!exec.buildSql) {
+ exec.stackPush(var);
+ }
+ else {
+ exec.stackPush(new Var(ident, Var.Type.STRING, var.toSqlString()));
+ }
+ }
+ else {
+ exec.stackPush(new Var(Var.Type.IDENT, ident));
+ }
+ return 0;
+ }
+
+ /**
+ * Single quoted string literal
+ */
+ @Override
+ public Integer visitSingle_quotedString(HplsqlParser.Single_quotedStringContext ctx) {
+ if (exec.buildSql) {
+ exec.stackPush(ctx.getText());
+ }
+ else {
+ exec.stackPush(Utils.unquoteString(ctx.getText()));
+ }
+ return 0;
+ }
+
+ /**
+ * Integer literal, signed or unsigned
+ */
+ @Override
+ public Integer visitInt_number(HplsqlParser.Int_numberContext ctx) {
+ exec.stack.push(new Var(new Long(ctx.getText())));
+ return 0;
+ }
+
+ /**
+ * Interval number (1 DAYS i.e)
+ */
+ @Override
+ public Integer visitInterval_number(HplsqlParser.Interval_numberContext ctx) {
+ int num = evalPop(ctx.int_number()).intValue();
+ Interval interval = new Interval().set(num, ctx.interval_item().getText());
+ stackPush(new Var(interval));
+ return 0;
+ }
+
+ /**
+ * Decimal literal, signed or unsigned
+ */
+ @Override
+ public Integer visitDec_number(HplsqlParser.Dec_numberContext ctx) {
+ stackPush(new Var(new BigDecimal(ctx.getText())));
+ return 0;
+ }
+
+ /**
+ * NULL constant
+ */
+ @Override
+ public Integer visitNull_const(HplsqlParser.Null_constContext ctx) {
+ stackPush(new Var());
+ return 0;
+ }
+
+ /**
+ * DATE 'YYYY-MM-DD' literal
+ */
+ @Override
+ public Integer visitDate_literal(HplsqlParser.Date_literalContext ctx) {
+ String str = evalPop(ctx.string()).toString();
+ stackPush(new Var(Var.Type.DATE, Utils.toDate(str)));
+ return 0;
+ }
+
+ /**
+ * TIMESTAMP 'YYYY-MM-DD HH:MI:SS.FFF' literal
+ */
+ @Override
+ public Integer visitTimestamp_literal(HplsqlParser.Timestamp_literalContext ctx) {
+ String str = evalPop(ctx.string()).toString();
+ int len = str.length();
+ int precision = 0;
+ if (len > 19 && len <= 29) {
+ precision = len - 20;
+ if (precision > 3) {
+ precision = 3;
+ }
+ }
+ stackPush(new Var(Utils.toTimestamp(str), precision));
+ return 0;
+ }
+
+ /**
+ * Define the connection profile to execute the current statement
+ */
+ String getStatementConnection() {
+ if (exec.stmtConnList.contains(exec.conf.defaultConnection)) {
+ return exec.conf.defaultConnection;
+ }
+ else if (!exec.stmtConnList.isEmpty()) {
+ return exec.stmtConnList.get(0);
+ }
+ return exec.conf.defaultConnection;
+ }
+
+ /**
+ * Define the connection profile for the specified object
+ * @return
+ */
+ String getObjectConnection(String name) {
+ String conn = exec.objectConnMap.get(name.toUpperCase());
+ if (conn != null) {
+ return conn;
+ }
+ return exec.conf.defaultConnection;
+ }
+
+ /**
+ * Get the connection (open the new connection if not available)
+ * @throws Exception
+ */
+ Connection getConnection(String conn) throws Exception {
+ return exec.conn.getConnection(conn);
+ }
+
+ /**
+ * Return the connection to the pool
+ */
+ void returnConnection(String name, Connection conn) {
+ exec.conn.returnConnection(name, conn);
+ }
+
+ /**
+ * Define the database type by profile name
+ */
+ Conn.Type getConnectionType(String conn) {
+ return exec.conn.getType(conn);
+ }
+
+ /**
+ * Get the current database type
+ */
+ public Conn.Type getConnectionType() {
+ return getConnectionType(exec.conf.defaultConnection);
+ }
+
+ /**
+ * Add managed temporary table
+ */
+ public void addManagedTable(String name, String managedName) {
+ exec.managedTables.put(name, managedName);
+ }
+
+ /**
+ * Get node text including spaces
+ */
+ String getText(ParserRuleContext ctx) {
+ return ctx.start.getInputStream().getText(new org.antlr.v4.runtime.misc.Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+ }
+
+ String getText(ParserRuleContext ctx, Token start, Token stop) {
+ return ctx.start.getInputStream().getText(new org.antlr.v4.runtime.misc.Interval(start.getStartIndex(), stop.getStopIndex()));
+ }
+
+ /**
+ * Evaluate the expression and pop value from the stack
+ */
+ Var evalPop(ParserRuleContext ctx) {
+ visit(ctx);
+ if (!exec.stack.isEmpty()) {
+ return exec.stackPop();
+ }
+ return Var.Empty;
+ }
+
+ Var evalPop(ParserRuleContext ctx, long def) {
+ visit(ctx);
+ if (!exec.stack.isEmpty()) {
+ return stackPop();
+ }
+ return new Var(def);
+ }
+
+ /**
+ * Evaluate the data type and length
+ *
+ */
+ String evalPop(HplsqlParser.DtypeContext type, HplsqlParser.Dtype_lenContext len) {
+ if (isConvert(exec.conf.defaultConnection)) {
+ return exec.converter.dataType(type, len);
+ }
+ return getText(type, type.getStart(), len.getStop());
+ }
+
+ /**
+ * Evaluate the expression to NULL
+ */
+ void evalNull() {
+ stackPush(Var.Null);
+ }
+
+ /**
+ * Get formatted text between 2 tokens
+ */
+ public String getFormattedText(ParserRuleContext ctx) {
+ return ctx.start.getInputStream().getText(
+ new org.antlr.v4.runtime.misc.Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+ }
+
+ /**
+ * Flag whether executed from UDF or not
+ */
+ void setUdfRun(boolean udfRun) {
+ this.udfRun = udfRun;
+ }
+
+ /**
+ * Whether on-the-fly SQL conversion is required for the connection
+ */
+ boolean isConvert(String connName) {
+ return exec.conf.getConnectionConvert(connName);
+ }
+
+ /**
+ * Increment the row count
+ */
+ public int incRowCount() {
+ return exec.rowCount++;
+ }
+
+ /**
+ * Set the row count
+ */
+ public void setRowCount(int rowCount) {
+ exec.rowCount = rowCount;
+ }
+
+ /**
+ * Trace information
+ */
+ public void trace(ParserRuleContext ctx, String message) {
+ if (!trace) {
+ return;
+ }
+ if (ctx != null) {
+ System.out.println("Ln:" + ctx.getStart().getLine() + " " + message);
+ }
+ else {
+ System.out.println(message);
+ }
+ }
+
+ /**
+ * Informational messages
+ */
+ public void info(ParserRuleContext ctx, String message) {
+ if (!info) {
+ return;
+ }
+ if (ctx != null) {
+ System.err.println("Ln:" + ctx.getStart().getLine() + " " + message);
+ }
+ else {
+ System.err.println(message);
+ }
+ }
+
+ public Stack<Var> getStack() {
+ return exec.stack;
+ }
+
+ public int getRowCount() {
+ return exec.rowCount;
+ }
+
+ public Conf getConf() {
+ return exec.conf;
+ }
+
+ public boolean getTrace() {
+ return exec.trace;
+ }
+
+ public boolean getInfo() {
+ return exec.info;
+ }
+
+ public boolean getOffline() {
+ return exec.offline;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
new file mode 100644
index 0000000..f8b01e1
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Expression.java
@@ -0,0 +1,574 @@
+/**
+ * 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.hive.hplsql;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Calendar;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hive.hplsql.Var.Type;
+
+/**
+ * Expressions
+ */
+public class Expression {
+
+ Exec exec;
+ boolean trace = false;
+
+ Expression(Exec e) {
+ exec = e;
+ trace = exec.getTrace();
+ }
+
+ /**
+ * Evaluate an expression
+ */
+ public void exec(HplsqlParser.ExprContext ctx) {
+ if (ctx.T_ADD() != null) {
+ operatorAdd(ctx);
+ }
+ else if (ctx.T_SUB() != null) {
+ operatorSub(ctx);
+ }
+ else if (ctx.interval_item() != null) {
+ createInterval(ctx);
+ }
+ else {
+ visitChildren(ctx);
+ }
+ }
+
+ /**
+ * Evaluate an expression in executable SQL statement
+ */
+ public void execSql(HplsqlParser.ExprContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ if (ctx.T_OPEN_P() != null) {
+ sql.append("(");
+ sql.append(evalPop(ctx.expr(0)).toString());
+ sql.append(")");
+ }
+ else if (ctx.T_ADD() != null) {
+ sql.append(evalPop(ctx.expr(0)).toString());
+ sql.append(" + ");
+ sql.append(evalPop(ctx.expr(1)).toString());
+ }
+ else if (ctx.T_SUB() != null) {
+ sql.append(evalPop(ctx.expr(0)).toString());
+ sql.append(" - ");
+ sql.append(evalPop(ctx.expr(1)).toString());
+ }
+ else if (ctx.interval_item() != null) {
+ sql.append(exec.getFormattedText(ctx));
+ }
+ else {
+ visitChildren(ctx);
+ sql.append(exec.stackPop().toString());
+ }
+ exec.stackPush(sql);
+ }
+
+ /**
+ * Evaluate a boolean expression
+ */
+ public void execBool(HplsqlParser.Bool_exprContext ctx) {
+ if (ctx.T_OPEN_P() != null) {
+ eval(ctx.bool_expr(0));
+ return;
+ }
+ else if (ctx.bool_expr_atom() != null) {
+ eval(ctx.bool_expr_atom());
+ return;
+ }
+ Var result = evalPop(ctx.bool_expr(0));
+ if (ctx.bool_expr_logical_operator() != null) {
+ if (ctx.bool_expr_logical_operator().T_AND() != null) {
+ if (result.isTrue()) {
+ result = evalPop(ctx.bool_expr(1));
+ }
+ }
+ else if (ctx.bool_expr_logical_operator().T_OR() != null) {
+ if (!result.isTrue()) {
+ result = evalPop(ctx.bool_expr(1));
+ }
+ }
+ }
+ exec.stackPush(result);
+ }
+
+ /**
+ * Evaluate a boolean expression in executable SQL statement
+ */
+ public void execBoolSql(HplsqlParser.Bool_exprContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ if (ctx.T_OPEN_P() != null) {
+ sql.append("(");
+ sql.append(evalPop(ctx.bool_expr(0)).toString());
+ sql.append(")");
+ }
+ else if (ctx.bool_expr_atom() != null) {
+ sql.append(evalPop(ctx.bool_expr_atom()).toString());
+ }
+ else if (ctx.bool_expr_logical_operator() != null) {
+ sql.append(evalPop(ctx.bool_expr(0)).toString());
+ sql.append(" " + ctx.bool_expr_logical_operator().getText() + " ");
+ sql.append(evalPop(ctx.bool_expr(1)).toString());
+ }
+ exec.stackPush(sql);
+ }
+
+ /**
+ * Binary boolean expression
+ */
+ public Integer execBoolBinary(HplsqlParser.Bool_expr_binaryContext ctx) {
+ HplsqlParser.Bool_expr_binary_operatorContext op = ctx.bool_expr_binary_operator();
+ if (op.T_EQUAL() != null || op.T_EQUAL2() != null) {
+ operatorEqual(ctx, true);
+ }
+ else if (op.T_NOTEQUAL() != null || op.T_NOTEQUAL2() != null) {
+ operatorEqual(ctx, false);
+ }
+ else if (op.T_GREATER() != null || op.T_LESS() != null || op.T_GREATEREQUAL() != null || op.T_LESSEQUAL() != null) {
+ operatorCompare(ctx, op);
+ }
+ else {
+ exec.stackPush(false);
+ }
+ return 0;
+ }
+
+ /**
+ * Binary boolean expression in executable SQL statement
+ */
+ public Integer execBoolBinarySql(HplsqlParser.Bool_expr_binaryContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append(evalPop(ctx.expr(0)).toString());
+ sql.append(" " + exec.getFormattedText(ctx.bool_expr_binary_operator()) + " ");
+ sql.append(evalPop(ctx.expr(1)).toString());
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * Unary boolean expression
+ */
+ public Integer execBoolUnary(HplsqlParser.Bool_expr_unaryContext ctx) {
+ boolean val = false;
+ if (ctx.T_IS() != null) {
+ val = evalPop(ctx.expr(0)).isNull();
+ if (ctx.T_NOT() != null) {
+ val = !val;
+ }
+ }
+ else if (ctx.T_BETWEEN() != null) {
+ Var v = evalPop(ctx.expr(0));
+ Var v1 = evalPop(ctx.expr(1));
+ int cmp = v.compareTo(v1);
+ if (cmp >= 0) {
+ Var v2 = evalPop(ctx.expr(2));
+ cmp = v.compareTo(v2);
+ if (cmp <= 0) {
+ val = true;
+ }
+ }
+ }
+ exec.stackPush(val);
+ return 0;
+ }
+
+ /**
+ * Unary boolean expression in executable SQL statement
+ */
+ public Integer execBoolUnarySql(HplsqlParser.Bool_expr_unaryContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ if (ctx.T_IS() != null) {
+ sql.append(evalPop(ctx.expr(0)).toString());
+ sql.append(" " + exec.getText(ctx, ctx.T_IS().getSymbol(), ctx.T_NULL().getSymbol()));
+ }
+ else if (ctx.T_BETWEEN() != null) {
+ sql.append(evalPop(ctx.expr(0)).toString());
+ sql.append(" " + ctx.T_BETWEEN().getText() + " ");
+ sql.append(evalPop(ctx.expr(1)).toString());
+ sql.append(" " + ctx.T_AND().getText() + " ");
+ sql.append(evalPop(ctx.expr(2)).toString());
+ }
+ else if (ctx.bool_expr_single_in() != null) {
+ singleInClauseSql(ctx.bool_expr_single_in(), sql);
+ }
+ else if (ctx.bool_expr_multi_in() != null) {
+ multiInClauseSql(ctx.bool_expr_multi_in(), sql);
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * Single value IN clause in executable SQL statement
+ */
+ public void singleInClauseSql(HplsqlParser.Bool_expr_single_inContext ctx, StringBuilder sql) {
+ sql.append(evalPop(ctx.expr(0)).toString());
+ if (ctx.T_NOT() != null) {
+ sql.append(" " + ctx.T_NOT().getText());
+ }
+ sql.append(" " + ctx.T_IN().getText() + " (");
+ if (ctx.select_stmt() != null) {
+ sql.append(evalPop(ctx.select_stmt()));
+ }
+ else {
+ int cnt = ctx.expr().size();
+ for (int i = 1; i < cnt; i++) {
+ sql.append(evalPop(ctx.expr(i)).toString());
+ if (i + 1 < cnt) {
+ sql.append(", ");
+ }
+ }
+ }
+ sql.append(")");
+ }
+
+ /**
+ * Multi-value IN clause in executable SQL statement
+ */
+ public void multiInClauseSql(HplsqlParser.Bool_expr_multi_inContext ctx, StringBuilder sql) {
+ int cnt = ctx.expr().size();
+ sql.append("(");
+ for (int i = 0; i < cnt; i++) {
+ sql.append(evalPop(ctx.expr(i)).toString());
+ if (i + 1 < cnt) {
+ sql.append(", ");
+ }
+ }
+ sql.append(")");
+ if (ctx.T_NOT() != null) {
+ sql.append(" " + ctx.T_NOT().getText());
+ }
+ sql.append(" " + ctx.T_IN().getText() + " (");
+ if (ctx.select_stmt() != null) {
+ sql.append(evalPop(ctx.select_stmt()));
+ }
+ sql.append(")");
+ }
+
+ /**
+ * Addition operator
+ */
+ public void operatorAdd(HplsqlParser.ExprContext ctx) {
+ Var v1 = evalPop(ctx.expr(0));
+ Var v2 = evalPop(ctx.expr(1));
+ if (v1.value == null || v2.value == null) {
+ evalNull();
+ }
+ else if (v1.type == Type.BIGINT && v2.type == Type.BIGINT) {
+ exec.stackPush(new Var((Long)v1.value + (Long)v2.value));
+ }
+ else if (v1.type == Type.BIGINT && v2.type == Type.DATE) {
+ exec.stackPush(changeDateByInt((Date)v2.value, (Long)v1.value, true /*add*/));
+ }
+ else if (v1.type == Type.DATE && v2.type == Type.BIGINT) {
+ exec.stackPush(changeDateByInt((Date)v1.value, (Long)v2.value, true /*add*/));
+ }
+ else if (v1.type == Type.DATE && v2.type == Type.INTERVAL) {
+ exec.stackPush(new Var(((Interval)v2.value).dateChange((Date)v1.value, true /*add*/)));
+ }
+ else if (v1.type == Type.TIMESTAMP && v2.type == Type.INTERVAL) {
+ exec.stackPush(new Var(((Interval)v2.value).timestampChange((Timestamp)v1.value, true /*add*/), v1.scale));
+ }
+ }
+
+ /**
+ * Subtraction operator
+ */
+ public void operatorSub(HplsqlParser.ExprContext ctx) {
+ Var v1 = evalPop(ctx.expr(0));
+ Var v2 = evalPop(ctx.expr(1));
+ if (v1.value == null || v2.value == null) {
+ evalNull();
+ }
+ else if (v1.type == Type.BIGINT && v2.type == Type.BIGINT) {
+ exec.stackPush(new Var((Long)v1.value - (Long)v2.value));
+ }
+ else if (v1.type == Type.DATE && v2.type == Type.BIGINT) {
+ exec.stackPush(changeDateByInt((Date)v1.value, (Long)v2.value, false /*subtract*/));
+ }
+ else if (v1.type == Type.DATE && v2.type == Type.INTERVAL) {
+ exec.stackPush(new Var(((Interval)v2.value).dateChange((Date)v1.value, false /*subtract*/)));
+ }
+ else if (v1.type == Type.TIMESTAMP && v2.type == Type.INTERVAL) {
+ exec.stackPush(new Var(((Interval)v2.value).timestampChange((Timestamp)v1.value, false /*subtract*/), v1.scale));
+ }
+ }
+
+ /**
+ * Add or subtract the specified number of days from DATE
+ */
+ public Var changeDateByInt(Date d, Long i, boolean add) {
+ Calendar c = Calendar.getInstance();
+ c.setTimeInMillis(d.getTime());
+ int days = i.intValue();
+ if(!add) {
+ days *= -1;
+ }
+ c.add(Calendar.DAY_OF_MONTH, days);
+ return new Var(new Date(c.getTimeInMillis()));
+ }
+
+ /**
+ * Equality operator
+ */
+ public void operatorEqual(HplsqlParser.Bool_expr_binaryContext ctx, boolean equal) {
+ Var v1 = evalPop(ctx.expr(0));
+ Var v2 = evalPop(ctx.expr(1));
+ boolean eq = v1.equals(v2);
+ if (!equal) {
+ eq = !eq;
+ }
+ exec.stackPush(eq);
+ }
+
+ /**
+ * Comparison operator
+ */
+ public void operatorCompare(HplsqlParser.Bool_expr_binaryContext ctx, HplsqlParser.Bool_expr_binary_operatorContext op) {
+ Var v1 = evalPop(ctx.expr(0));
+ Var v2 = evalPop(ctx.expr(1));
+ int cmp = v1.compareTo(v2);
+ boolean bool = false;
+ if (op.T_GREATER() != null) {
+ if (cmp > 0) {
+ bool = true;
+ }
+ }
+ else if (op.T_GREATEREQUAL() != null) {
+ if (cmp >= 0) {
+ bool = true;
+ }
+ }
+ if (op.T_LESS() != null) {
+ if (cmp < 0) {
+ bool = true;
+ }
+ }
+ else if (op.T_LESSEQUAL() != null) {
+ if (cmp <= 0) {
+ bool = true;
+ }
+ }
+ exec.stackPush(bool);
+ }
+
+ /**
+ * String concatenation operator
+ */
+ public void operatorConcat(HplsqlParser.Expr_concatContext ctx) {
+ StringBuilder val = new StringBuilder();
+ int cnt = ctx.expr_concat_item().size();
+ boolean nulls = true;
+ for (int i = 0; i < cnt; i++) {
+ Var c = evalPop(ctx.expr_concat_item(i));
+ if (!c.isNull()) {
+ val.append(c.toString());
+ nulls = false;
+ }
+ }
+ if (nulls) {
+ evalNull();
+ }
+ else {
+ evalString(val);
+ }
+ }
+
+ /**
+ * String concatenation operator in executable SQL statement
+ */
+ public void operatorConcatSql(HplsqlParser.Expr_concatContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append("CONCAT(");
+ int cnt = ctx.expr_concat_item().size();
+ for (int i = 0; i < cnt; i++) {
+ sql.append(evalPop(ctx.expr_concat_item(i)).toString());
+ if (i + 1 < cnt) {
+ sql.append(", ");
+ }
+ }
+ sql.append(")");
+ exec.stackPush(sql);
+ }
+
+ /**
+ * Simple CASE expression
+ */
+ public void execSimpleCase(HplsqlParser.Expr_case_simpleContext ctx) {
+ int i = 1;
+ int cnt = ctx.expr().size();
+ boolean found = false;
+ Var val = evalPop(ctx.expr(0));
+ while(i < cnt) {
+ Var when = evalPop(ctx.expr(i));
+ if(val.compareTo(when) == 0) {
+ visit(ctx.expr(i + 1));
+ found = true;
+ break;
+ }
+ i += 2;
+ }
+ if(!found) {
+ if(ctx.T_ELSE() != null) {
+ visit(ctx.expr(cnt - 1));
+ }
+ else {
+ evalNull();
+ }
+ }
+ }
+
+ /**
+ * Simple CASE expression in executable SQL statement
+ */
+ public void execSimpleCaseSql(HplsqlParser.Expr_case_simpleContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append("CASE ");
+ sql.append(evalPop(ctx.expr(0)).toString());
+ int cnt = ctx.T_WHEN().size();
+ for (int i = 0; i < cnt; i++) {
+ sql.append(" WHEN ");
+ sql.append(evalPop(ctx.expr(i * 2 + 1)).toString());
+ sql.append(" THEN ");
+ sql.append(evalPop(ctx.expr(i * 2 + 2)).toString());
+ }
+ if (ctx.T_ELSE() != null) {
+ sql.append(" ELSE ");
+ sql.append(evalPop(ctx.expr(cnt * 2 + 1)).toString());
+ }
+ sql.append(" END");
+ exec.stackPush(sql);
+ }
+
+ /**
+ * Searched CASE expression
+ */
+ public void execSearchedCase(HplsqlParser.Expr_case_searchedContext ctx) {
+ int cnt = ctx.bool_expr().size();
+ boolean found = false;
+ for(int i = 0; i < cnt; i++) {
+ if(evalPop(ctx.bool_expr(i)).isTrue()) {
+ visit(ctx.expr(i));
+ found = true;
+ break;
+ }
+ }
+ if(!found) {
+ if(ctx.T_ELSE() != null) {
+ visit(ctx.expr(cnt));
+ }
+ else {
+ evalNull();
+ }
+ }
+ }
+
+ /**
+ * Searched CASE expression in executable SQL statement
+ */
+ public void execSearchedCaseSql(HplsqlParser.Expr_case_searchedContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append("CASE");
+ int cnt = ctx.T_WHEN().size();
+ for (int i = 0; i < cnt; i++) {
+ sql.append(" WHEN ");
+ sql.append(evalPop(ctx.bool_expr(i)).toString());
+ sql.append(" THEN ");
+ sql.append(evalPop(ctx.expr(i)).toString());
+ }
+ if (ctx.T_ELSE() != null) {
+ sql.append(" ELSE ");
+ sql.append(evalPop(ctx.expr(cnt)).toString());
+ }
+ sql.append(" END");
+ exec.stackPush(sql);
+ }
+
+ /**
+ * Create an interval variable
+ */
+ public void createInterval(HplsqlParser.ExprContext ctx) {
+ int num = evalPop(ctx.expr(0)).intValue();
+ Interval interval = new Interval().set(num, ctx.interval_item().getText());
+ exec.stackPush(new Var(interval));
+ }
+
+ /**
+ * Evaluate the expression and push the value to the stack
+ */
+ void eval(ParserRuleContext ctx) {
+ visit(ctx);
+ }
+
+ /**
+ * Evaluate the expression and pop value from the stack
+ */
+ Var evalPop(ParserRuleContext ctx) {
+ visit(ctx);
+ if (!exec.stack.isEmpty()) {
+ return exec.stackPop();
+ }
+ return Var.Empty;
+ }
+
+ /**
+ * Evaluate the expression to specified String value
+ */
+ void evalString(String string) {
+ exec.stackPush(new Var(string));
+ }
+
+ void evalString(StringBuilder string) {
+ evalString(string.toString());
+ }
+
+ /**
+ * Evaluate the expression to NULL
+ */
+ void evalNull() {
+ exec.stackPush(Var.Null);
+ }
+
+ /**
+ * Execute rules
+ */
+ Integer visit(ParserRuleContext ctx) {
+ return exec.visit(ctx);
+ }
+
+ /**
+ * Execute children rules
+ */
+ Integer visitChildren(ParserRuleContext ctx) {
+ return exec.visitChildren(ctx);
+ }
+
+ /**
+ * Trace information
+ */
+ public void trace(ParserRuleContext ctx, String message) {
+ exec.trace(ctx, message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/File.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/File.java b/hplsql/src/main/java/org/apache/hive/hplsql/File.java
new file mode 100644
index 0000000..6a8ddfe
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/File.java
@@ -0,0 +1,132 @@
+/**
+ * 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.hive.hplsql;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * HDFS file operations
+ */
+public class File {
+ Path path;
+ FileSystem fs;
+ FSDataInputStream in;
+ FSDataOutputStream out;
+
+ /**
+ * Create FileSystem object
+ */
+ public FileSystem createFs() throws IOException {
+ fs = FileSystem.get(new Configuration());
+ return fs;
+ }
+
+ /**
+ * Create a file
+ */
+ public void create(String dir, String file, boolean overwrite) {
+ path = new Path(dir, file);
+ try {
+ if (fs == null) {
+ fs = FileSystem.get(new Configuration());
+ }
+ out = fs.create(path, overwrite);
+ }
+ catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * Open an existing file
+ */
+ public void open(String dir, String file) {
+ path = new Path(dir, file);
+ try {
+ if (fs == null) {
+ fs = FileSystem.get(new Configuration());
+ }
+ in = fs.open(path);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * Read a character from input
+ * @throws IOException
+ */
+ public char readChar() throws IOException {
+ return in.readChar();
+ }
+
+ /**
+ * Write string to file
+ */
+ public void writeString(String str) {
+ try {
+ out.writeChars(str);
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * Close a file
+ */
+ public void close() {
+ try {
+ if (in != null) {
+ in.close();
+ }
+ if (out != null) {
+ out.close();
+ }
+ in = null;
+ out = null;
+ path = null;
+ fs = null;
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * Get the fully-qualified path
+ * NOTE: FileSystem.resolvePath() is not available in Hadoop 1.2.1
+ * @throws IOException
+ */
+ public Path resolvePath(Path path) throws IOException {
+ return fs.getFileStatus(path).getPath();
+ }
+
+ @Override
+ public String toString() {
+ if (path != null) {
+ return "FILE <" + path.toString() + ">";
+ }
+ return "FILE <null>";
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Handler.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Handler.java b/hplsql/src/main/java/org/apache/hive/hplsql/Handler.java
new file mode 100644
index 0000000..6c292ef
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Handler.java
@@ -0,0 +1,41 @@
+/**
+ * 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.hive.hplsql;
+
+import org.apache.hive.hplsql.Signal.Type;
+
+/**
+ * HPL/SQL condition and exception handler
+ */
+public class Handler {
+ public enum ExecType { CONTINUE, EXIT };
+ ExecType execType;
+ Type type;
+ String value;
+ Scope scope;
+ HplsqlParser.Declare_handler_itemContext ctx;
+
+ Handler(ExecType execType, Type type, String value, Scope scope, HplsqlParser.Declare_handler_itemContext ctx) {
+ this.execType = execType;
+ this.type = type;
+ this.value = value;
+ this.scope = scope;
+ this.ctx = ctx;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Hplsql.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Hplsql.java b/hplsql/src/main/java/org/apache/hive/hplsql/Hplsql.java
new file mode 100644
index 0000000..c2545f7
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Hplsql.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.hplsql;
+
+public class Hplsql {
+ public static void main(String[] args) throws Exception {
+ System.exit(new Exec().run(args));
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Interval.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Interval.java b/hplsql/src/main/java/org/apache/hive/hplsql/Interval.java
new file mode 100644
index 0000000..92c5d52
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Interval.java
@@ -0,0 +1,109 @@
+/**
+ * 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.hive.hplsql;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Calendar;
+
+/**
+ * Date and time interval
+ */
+public class Interval {
+ int days = 0;
+ int milliseconds = 0;
+
+ /**
+ * Add or subtract interval value to the specified date
+ */
+ public Date dateChange(Date in, boolean add) {
+ Calendar c = Calendar.getInstance();
+ c.setTimeInMillis(in.getTime());
+ calendarChange(c, add);
+ return new Date(c.getTimeInMillis());
+ }
+
+ /**
+ * Add or subtract interval value to the specified timestamp
+ */
+ public Timestamp timestampChange(Timestamp in, boolean add) {
+ Calendar c = Calendar.getInstance();
+ c.setTimeInMillis(in.getTime());
+ calendarChange(c, add);
+ return new Timestamp(c.getTimeInMillis());
+ }
+
+ /**
+ * Add interval value to the specified Calendar value
+ */
+ public Calendar calendarChange(Calendar c, boolean add) {
+ int a = 1;
+ if (!add) {
+ a = -1;
+ }
+ if (days != 0) {
+ c.add(Calendar.DAY_OF_MONTH, days * a);
+ }
+ if (milliseconds != 0) {
+ c.setTimeInMillis(c.getTimeInMillis() + milliseconds * a);
+ }
+ return c;
+ }
+
+ /**
+ * Set interval value
+ */
+ public Interval set(int value, String item) {
+ if (item.compareToIgnoreCase("DAYS") == 0 || item.compareToIgnoreCase("DAY") == 0) {
+ setDays(value);
+ }
+ if (item.compareToIgnoreCase("MICROSECONDS") == 0 || item.compareToIgnoreCase("MICROSECOND") == 0) {
+ setMilliseconds(value);
+ }
+ return this;
+ }
+
+ /**
+ * Set interval items
+ */
+ public void setDays(int days) {
+ this.days = days;
+ }
+
+ public void setMilliseconds(int milliseconds) {
+ this.milliseconds = milliseconds;
+ }
+
+ /**
+ * Convert interval to string
+ */
+ @Override
+ public String toString() {
+ StringBuilder s = new StringBuilder();
+ if (days != 0) {
+ s.append(days);
+ s.append(" days");
+ }
+ if (milliseconds != 0) {
+ s.append(milliseconds);
+ s.append(" milliseconds");
+ }
+ return s.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Query.java b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
new file mode 100644
index 0000000..23d963f
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Query.java
@@ -0,0 +1,155 @@
+/**
+ * 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.hive.hplsql;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+
+public class Query {
+ String sql;
+ ParserRuleContext sqlExpr;
+ ParserRuleContext sqlSelect;
+
+ Connection conn;
+ Statement stmt;
+ ResultSet rs;
+ Exception exception;
+
+ Query() {
+ }
+
+ Query(String sql) {
+ this.sql = sql;
+ }
+
+ /**
+ * Set query objects
+ */
+ public void set(Connection conn, Statement stmt, ResultSet rs) {
+ this.conn = conn;
+ this.stmt = stmt;
+ this.rs = rs;
+ }
+
+ /**
+ * Get the number of rows
+ */
+ public int getRowCount() {
+ if (!error() && stmt != null) {
+ try {
+ return stmt.getUpdateCount();
+ } catch (SQLException e) {}
+ }
+ return -1;
+ }
+
+ /**
+ * Close statement results
+ */
+ public void closeStatement() {
+ try {
+ if(rs != null) {
+ rs.close();
+ rs = null;
+ }
+ if(stmt != null) {
+ stmt.close();
+ stmt = null;
+ }
+ } catch (SQLException e) {
+ e.printStackTrace();
+ }
+ }
+
+ /**
+ * Set SQL statement
+ */
+ public void setSql(String sql) {
+ this.sql = sql;
+ }
+
+ /**
+ * Set expression context
+ */
+ public void setExprCtx(ParserRuleContext sqlExpr) {
+ this.sqlExpr = sqlExpr;
+ }
+
+ /**
+ * Set SELECT statement context
+ */
+ public void setSelectCtx(ParserRuleContext sqlSelect) {
+ this.sqlSelect = sqlSelect;
+ }
+
+ /**
+ * Set an execution error
+ */
+ public void setError(Exception e) {
+ exception = e;
+ }
+
+ /**
+ * Print error stack trace
+ */
+ public void printStackTrace() {
+ if(exception != null) {
+ exception.printStackTrace();
+ }
+ }
+
+ /**
+ * Get the result set object
+ */
+ public ResultSet getResultSet() {
+ return rs;
+ }
+
+ /**
+ * Get the connection object
+ */
+ public Connection getConnection() {
+ return conn;
+ }
+
+ /**
+ * Return error information
+ */
+ public boolean error() {
+ return exception != null;
+ }
+
+ public String errorText() {
+ if(exception != null) {
+ if(exception instanceof ClassNotFoundException) {
+ return "ClassNotFoundException: " + exception.getMessage();
+ }
+ return exception.getMessage();
+ }
+ return "";
+ }
+
+ public Exception getException() {
+ return exception;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Scope.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Scope.java b/hplsql/src/main/java/org/apache/hive/hplsql/Scope.java
new file mode 100644
index 0000000..317a94f
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Scope.java
@@ -0,0 +1,69 @@
+/**
+ * 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.hive.hplsql;
+
+import java.util.ArrayList;
+
+/**
+ * HPL/SQL block scope
+ */
+public class Scope {
+
+ // Types
+ public enum Type { FILE, BEGIN_END, LOOP, HANDLER, ROUTINE };
+
+ // Local variables
+ ArrayList<Var> vars = new ArrayList<Var>();
+ // Condition handlers
+ ArrayList<Handler> handlers = new ArrayList<Handler>();
+
+ Scope parent;
+ Type type;
+
+ Scope(Type type) {
+ this.parent = null;
+ this.type = type;
+ }
+
+ Scope(Scope parent, Type type) {
+ this.parent = parent;
+ this.type = type;
+ }
+
+ /**
+ * Add a local variable
+ */
+ void addVariable(Var var) {
+ vars.add(var);
+ }
+
+ /**
+ * Add a condition handler
+ */
+ void addHandler(Handler handler) {
+ handlers.add(handler);
+ }
+
+ /**
+ * Get the parent scope
+ */
+ Scope getParent() {
+ return parent;
+ }
+}
[15/25] hive git commit: HIVE-10796: Remove dependencies on
NumericHistogram and NumDistinctValueEstimator from JavaDataModel (Owen
O'Malley via Prasanth Jayachandran)
Posted by se...@apache.org.
HIVE-10796: Remove dependencies on NumericHistogram and NumDistinctValueEstimator from JavaDataModel (Owen O'Malley via Prasanth Jayachandran)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/20034d3e
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/20034d3e
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/20034d3e
Branch: refs/heads/llap
Commit: 20034d3ef223c249aa72ffa315238af6c7157afd
Parents: f755652
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Wed Jul 1 10:02:49 2015 -0700
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Wed Jul 1 10:02:49 2015 -0700
----------------------------------------------------------------------
.../ql/udf/generic/GenericUDAFComputeStats.java | 4 +--
.../generic/GenericUDAFHistogramNumeric.java | 2 +-
.../generic/GenericUDAFPercentileApprox.java | 2 +-
.../udf/generic/NumDistinctValueEstimator.java | 18 ++++++++++++
.../hive/ql/udf/generic/NumericHistogram.java | 14 +++++++++
.../hadoop/hive/ql/util/JavaDataModel.java | 30 --------------------
6 files changed, 36 insertions(+), 34 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/20034d3e/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
index 274330b..8482e18 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeStats.java
@@ -435,7 +435,7 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
public int estimate() {
JavaDataModel model = JavaDataModel.get();
return model.lengthFor(columnType) + model.primitive1() + model.primitive2() +
- model.lengthFor(numDV);
+ numDV.lengthFor(model);
}
protected void initNDVEstimator(int numBitVectors) {
@@ -833,7 +833,7 @@ public class GenericUDAFComputeStats extends AbstractGenericUDAFResolver {
public int estimate() {
JavaDataModel model = JavaDataModel.get();
return model.primitive1() * 2 + model.primitive2() * 4 +
- model.lengthFor(columnType) + model.lengthFor(numDV);
+ model.lengthFor(columnType) + numDV.lengthFor(model);
}
};
http://git-wip-us.apache.org/repos/asf/hive/blob/20034d3e/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
index 008d937..434956f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFHistogramNumeric.java
@@ -238,7 +238,7 @@ public class GenericUDAFHistogramNumeric extends AbstractGenericUDAFResolver {
NumericHistogram histogram; // the histogram object
@Override
public int estimate() {
- return JavaDataModel.get().lengthFor(histogram);
+ return histogram.lengthFor(JavaDataModel.get());
}
};
http://git-wip-us.apache.org/repos/asf/hive/blob/20034d3e/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentileApprox.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentileApprox.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentileApprox.java
index e53b893..89d95f8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentileApprox.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFPercentileApprox.java
@@ -358,7 +358,7 @@ public class GenericUDAFPercentileApprox extends AbstractGenericUDAFResolver {
@Override
public int estimate() {
JavaDataModel model = JavaDataModel.get();
- return model.lengthFor(histogram) +
+ return histogram.lengthFor(model) +
model.array() + JavaDataModel.PRIMITIVES2 * quantiles.length;
}
};
http://git-wip-us.apache.org/repos/asf/hive/blob/20034d3e/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumDistinctValueEstimator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumDistinctValueEstimator.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumDistinctValueEstimator.java
index 8212bea..7cc5734 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumDistinctValueEstimator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumDistinctValueEstimator.java
@@ -23,6 +23,7 @@ import javolution.util.FastBitSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
import org.apache.hadoop.io.Text;
public class NumDistinctValueEstimator {
@@ -356,4 +357,21 @@ public class NumDistinctValueEstimator {
numDistinctValues = Math.pow(2.0, avgLeastSigZero);
return ((long)(numDistinctValues));
}
+
+ public int lengthFor(JavaDataModel model) {
+ int length = model.object();
+ length += model.primitive1() * 2; // two int
+ length += model.primitive2(); // one double
+ length += model.lengthForRandom() * 2; // two Random
+
+ int numVector = getnumBitVectors();
+ if (numVector > 0) {
+ length += model.array() * 3; // three array
+ length += model.primitive1() * numVector * 2; // two int array
+ length += (model.object() + model.array() + model.primitive1() +
+ model.primitive2()) * numVector; // bitset array
+ }
+ return length;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/hive/blob/20034d3e/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumericHistogram.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumericHistogram.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumericHistogram.java
index 0577a4f..4182234 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumericHistogram.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/NumericHistogram.java
@@ -22,6 +22,7 @@ import java.util.List;
import java.util.Collections;
import java.util.Random;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
import org.apache.hadoop.hive.serde2.io.DoubleWritable;
import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
@@ -311,4 +312,17 @@ public class NumericHistogram {
public int getNumBins() {
return bins == null ? 0 : bins.size();
}
+
+ public int lengthFor(JavaDataModel model) {
+ int length = model.object();
+ length += model.primitive1() * 2; // two int
+ int numBins = getNumBins();
+ if (numBins > 0) {
+ length += model.arrayList(); // List<Coord>
+ // Coord holds two doubles
+ length += numBins * (model.object() + model.primitive2() * 2);
+ }
+ length += model.lengthForRandom(); // Random
+ return length;
+ }
}
http://git-wip-us.apache.org/repos/asf/hive/blob/20034d3e/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java b/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java
index 3352a08..a0cd188 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java
@@ -18,9 +18,6 @@
package org.apache.hadoop.hive.ql.util;
-import org.apache.hadoop.hive.ql.udf.generic.NumDistinctValueEstimator;
-import org.apache.hadoop.hive.ql.udf.generic.NumericHistogram;
-
/**
* Estimation of memory footprint of object
*/
@@ -160,33 +157,6 @@ public enum JavaDataModel {
return lengthForStringOfLength(string.length());
}
- public int lengthFor(NumericHistogram histogram) {
- int length = object();
- length += primitive1() * 2; // two int
- int numBins = histogram.getNumBins();
- if (numBins > 0) {
- length += arrayList(); // List<Coord>
- length += numBins * (object() + primitive2() * 2); // Coord holds two doubles
- }
- length += lengthForRandom(); // Random
- return length;
- }
-
- public int lengthFor(NumDistinctValueEstimator estimator) {
- int length = object();
- length += primitive1() * 2; // two int
- length += primitive2(); // one double
- length += lengthForRandom() * 2; // two Random
-
- int numVector = estimator.getnumBitVectors();
- if (numVector > 0) {
- length += array() * 3; // three array
- length += primitive1() * numVector * 2; // two int array
- length += (object() + array() + primitive1() + primitive2()) * numVector; // bitset array
- }
- return length;
- }
-
public int lengthForRandom() {
// boolean + double + AtomicLong
return object() + primitive1() + primitive2() + object() + primitive2();
[22/25] hive git commit: HIVE-11152 : Swapping join inputs in
ASTConverter (Jesus Camacho Rodriguez via John Pullokkaran)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out b/ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out
index c098105..9077fc0 100644
--- a/ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out
+++ b/ql/src/test/results/clientpositive/join_cond_pushdown_3.q.out
@@ -287,7 +287,7 @@ STAGE PLANS:
Processor Tree:
ListSink
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[15][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3
where p2.p_partkey = 1 and p3.p_name = p2.p_name
@@ -297,12 +297,12 @@ from part p1 join part p2 join part p3
where p2.p_partkey = 1 and p3.p_name = p2.p_name
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-2 is a root stage
- Stage-1 depends on stages: Stage-2
- Stage-0 depends on stages: Stage-1
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
STAGE PLANS:
- Stage: Stage-2
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -346,21 +346,22 @@ STAGE PLANS:
1 _col1 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-1
+ Stage: Stage-2
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Select Operator
@@ -371,11 +372,6 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- TableScan
- Reduce Output Operator
- sort order:
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
@@ -383,10 +379,10 @@ STAGE PLANS:
keys:
0
1
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), 1 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), 1 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
File Output Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join_cond_pushdown_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_cond_pushdown_4.q.out b/ql/src/test/results/clientpositive/join_cond_pushdown_4.q.out
index a1dd24e..55e7678 100644
--- a/ql/src/test/results/clientpositive/join_cond_pushdown_4.q.out
+++ b/ql/src/test/results/clientpositive/join_cond_pushdown_4.q.out
@@ -145,13 +145,13 @@ where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey
and p1.p_partkey = p2.p_partkey
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-3 is a root stage
- Stage-2 depends on stages: Stage-3
- Stage-1 depends on stages: Stage-2
- Stage-0 depends on stages: Stage-1
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-3
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -206,6 +206,13 @@ STAGE PLANS:
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col10 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col10 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -221,20 +228,13 @@ STAGE PLANS:
Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- TableScan
- Reduce Output Operator
- key expressions: _col10 (type: string)
- sort order: +
- Map-reduce partition columns: _col10 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col1 (type: string)
- 1 _col10 (type: string)
+ 0 _col10 (type: string)
+ 1 _col1 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -244,10 +244,17 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-1
+ Stage: Stage-3
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -263,33 +270,22 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- TableScan
- Reduce Output Operator
- key expressions: _col9 (type: int)
- sort order: +
- Map-reduce partition columns: _col9 (type: int)
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col9 (type: int)
+ 1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col27 (type: int), _col28 (type: string), _col29 (type: string), _col30 (type: string), _col31 (type: string), _col32 (type: int), _col33 (type: string), _col34 (type: double), _col35 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
+ File Output Operator
+ compressed: false
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
index 3fba77c..9775f30 100644
--- a/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/mapjoin_mapjoin.q.out
@@ -56,18 +56,18 @@ TOK_QUERY
STAGE DEPENDENCIES:
- Stage-6 is a root stage
- Stage-4 depends on stages: Stage-6
- Stage-0 depends on stages: Stage-4
+ Stage-7 is a root stage
+ Stage-5 depends on stages: Stage-7
+ Stage-0 depends on stages: Stage-5
STAGE PLANS:
- Stage: Stage-6
+ Stage: Stage-7
Map Reduce Local Work
Alias -> Map Local Tables:
$hdt$_0:src
Fetch Operator
limit: -1
- $hdt$_1:$hdt$_2:src1
+ $hdt$_2:src1
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
@@ -86,10 +86,10 @@ STAGE PLANS:
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- Position of Big Table: 1
- $hdt$_1:$hdt$_2:src1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ Position of Big Table: 0
+ $hdt$_2:src1
TableScan
alias: src1
Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -108,7 +108,7 @@ STAGE PLANS:
1 _col0 (type: string)
Position of Big Table: 0
- Stage: Stage-4
+ Stage: Stage-5
Map Reduce
Map Operator Tree:
TableScan
@@ -136,36 +136,32 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1
- Position of Big Table: 1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0
+ Position of Big Table: 0
Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string)
- outputColumnNames: _col0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- GlobalTableId: 0
-#### A masked pattern was here ####
- NumFilesPerFileSink: 1
- Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
-#### A masked pattern was here ####
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- properties:
- columns _col0
- columns.types string
- escape.delim \
- hive.serialization.extend.additional.nesting.levels true
- serialization.format 1
- serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- TotalFiles: 1
- GatherStats: false
- MultiFileSpray: false
+#### A masked pattern was here ####
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ columns _col0
+ columns.types string
+ escape.delim \
+ hive.serialization.extend.additional.nesting.levels true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -444,10 +440,10 @@ STAGE PLANS:
name: default.srcpart
name: default.srcpart
Truncated Path -> Alias:
- /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:$hdt$_1:srcpart]
- /srcpart/ds=2008-04-08/hr=12 [$hdt$_1:$hdt$_1:srcpart]
- /srcpart/ds=2008-04-09/hr=11 [$hdt$_1:$hdt$_1:srcpart]
- /srcpart/ds=2008-04-09/hr=12 [$hdt$_1:$hdt$_1:srcpart]
+ /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:srcpart]
+ /srcpart/ds=2008-04-08/hr=12 [$hdt$_1:srcpart]
+ /srcpart/ds=2008-04-09/hr=11 [$hdt$_1:srcpart]
+ /srcpart/ds=2008-04-09/hr=12 [$hdt$_1:srcpart]
Stage: Stage-0
Fetch Operator
@@ -462,18 +458,18 @@ POSTHOOK: query: explain
select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-6 is a root stage
- Stage-4 depends on stages: Stage-6
- Stage-0 depends on stages: Stage-4
+ Stage-7 is a root stage
+ Stage-5 depends on stages: Stage-7
+ Stage-0 depends on stages: Stage-5
STAGE PLANS:
- Stage: Stage-6
+ Stage: Stage-7
Map Reduce Local Work
Alias -> Map Local Tables:
$hdt$_0:src
Fetch Operator
limit: -1
- $hdt$_1:$hdt$_2:src1
+ $hdt$_2:src1
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
@@ -490,9 +486,9 @@ STAGE PLANS:
Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- $hdt$_1:$hdt$_2:src1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ $hdt$_2:src1
TableScan
alias: src1
Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -508,7 +504,7 @@ STAGE PLANS:
0 _col0 (type: string)
1 _col0 (type: string)
- Stage: Stage-4
+ Stage: Stage-5
Map Reduce
Map Operator Tree:
TableScan
@@ -533,21 +529,17 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0
Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string)
- outputColumnNames: _col0
+ File Output Operator
+ compressed: false
Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Local Work:
Map Reduce Local Work
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/auto_join13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_join13.q.out b/ql/src/test/results/clientpositive/spark/auto_join13.q.out
index c81d296..0ffe9e3 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join13.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join13.q.out
@@ -30,22 +30,22 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 1
+ Map 3
Map Operator Tree:
TableScan
alias: src
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((UDFToDouble(key) < 200.0) and UDFToDouble(key) is not null) (type: boolean)
- Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+ predicate: (UDFToDouble(key) < 100.0) (type: boolean)
+ Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
- 0 UDFToDouble(_col0) (type: double)
- 1 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
Local Work:
Map Reduce Local Work
Map 4
@@ -54,26 +54,26 @@ STAGE PLANS:
alias: src
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (UDFToDouble(key) < 100.0) (type: boolean)
- Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ predicate: ((UDFToDouble(key) < 200.0) and UDFToDouble(key) is not null) (type: boolean)
+ Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
+ 0 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ 1 UDFToDouble(_col0) (type: double)
Local Work:
Map Reduce Local Work
Stage: Stage-1
Spark
Edges:
- Reducer 3 <- Map 2 (GROUP, 1)
+ Reducer 2 <- Map 1 (GROUP, 1)
#### A masked pattern was here ####
Vertices:
- Map 2
+ Map 1
Map Operator Tree:
TableScan
alias: src
@@ -93,7 +93,7 @@ STAGE PLANS:
1 _col0 (type: string)
outputColumnNames: _col0, _col1, _col2
input vertices:
- 1 Map 4
+ 1 Map 3
Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (UDFToDouble(_col2) + UDFToDouble(_col0)) is not null (type: boolean)
@@ -102,14 +102,14 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 UDFToDouble(_col0) (type: double)
- 1 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
- outputColumnNames: _col2, _col3
+ 0 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ 1 UDFToDouble(_col0) (type: double)
+ outputColumnNames: _col1, _col2
input vertices:
- 0 Map 1
+ 1 Map 4
Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: hash(_col3,_col2) (type: int)
+ expressions: hash(_col2,_col1) (type: int)
outputColumnNames: _col0
Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
Group By Operator
@@ -123,7 +123,7 @@ STAGE PLANS:
value expressions: _col0 (type: bigint)
Local Work:
Map Reduce Local Work
- Reducer 3
+ Reducer 2
Reduce Operator Tree:
Group By Operator
aggregations: sum(VALUE._col0)
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out b/ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out
index 72f60d0..a234ff5 100644
--- a/ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out
+++ b/ql/src/test/results/clientpositive/spark/auto_join_without_localtask.q.out
@@ -324,9 +324,9 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 5 <- Map 4 (PARTITION-LEVEL SORT, 2), Map 6 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 5 (PARTITION-LEVEL SORT, 2)
- Reducer 3 <- Reducer 2 (SORT, 1)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2)
+ Reducer 4 <- Reducer 3 (SORT, 1)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -335,23 +335,6 @@ STAGE PLANS:
alias: a
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: value is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: value (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Map 4
- Map Operator Tree:
- TableScan
- alias: a
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
predicate: ((UDFToDouble(key) > 100.0) and value is not null) (type: boolean)
Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
Select Operator
@@ -364,7 +347,7 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: string)
- Map 6
+ Map 5
Map Operator Tree:
TableScan
alias: a
@@ -381,6 +364,23 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: a
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: value (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Reducer 2
Reduce Operator Tree:
Join Operator
@@ -388,18 +388,30 @@ STAGE PLANS:
Inner Join 0 to 1
keys:
0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col2
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string), _col2 (type: string)
- outputColumnNames: _col0, _col1
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: string)
+ sort order: ++
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: string), _col1 (type: string)
- sort order: ++
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- Reducer 3
+ Reducer 4
Reduce Operator Tree:
Select Operator
expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -415,22 +427,6 @@ STAGE PLANS:
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 5
- Reduce Operator Tree:
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col1 (type: string)
- sort order: +
- Map-reduce partition columns: _col1 (type: string)
- Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string)
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join13.q.out b/ql/src/test/results/clientpositive/spark/join13.q.out
index 3eb7720..4a045f7 100644
--- a/ql/src/test/results/clientpositive/spark/join13.q.out
+++ b/ql/src/test/results/clientpositive/spark/join13.q.out
@@ -32,8 +32,8 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -42,23 +42,6 @@ STAGE PLANS:
alias: src
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: ((UDFToDouble(key) < 200.0) and UDFToDouble(key) is not null) (type: boolean)
- Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: key (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: UDFToDouble(_col0) (type: double)
- sort order: +
- Map-reduce partition columns: UDFToDouble(_col0) (type: double)
- Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
- Map 3
- Map Operator Tree:
- TableScan
- alias: src
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
predicate: (UDFToDouble(key) < 100.0) (type: boolean)
Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
Select Operator
@@ -71,7 +54,7 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: string)
- Map 5
+ Map 4
Map Operator Tree:
TableScan
alias: src
@@ -88,33 +71,29 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: ((UDFToDouble(key) < 200.0) and UDFToDouble(key) is not null) (type: boolean)
+ Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: key (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: UDFToDouble(_col0) (type: double)
+ sort order: +
+ Map-reduce partition columns: UDFToDouble(_col0) (type: double)
+ Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
Reducer 2
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 UDFToDouble(_col0) (type: double)
- 1 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
- outputColumnNames: _col2, _col3
- Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col3 (type: string), _col2 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 4
- Reduce Operator Tree:
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
0 _col0 (type: string)
1 _col0 (type: string)
outputColumnNames: _col0, _col1, _col2
@@ -128,6 +107,27 @@ STAGE PLANS:
Map-reduce partition columns: (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: string), _col2 (type: string)
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ 1 UDFToDouble(_col0) (type: double)
+ outputColumnNames: _col1, _col2
+ Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col2 (type: string), _col1 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join32.q.out b/ql/src/test/results/clientpositive/spark/join32.q.out
index d7383fb..4ae9dc6 100644
--- a/ql/src/test/results/clientpositive/spark/join32.q.out
+++ b/ql/src/test/results/clientpositive/spark/join32.q.out
@@ -110,25 +110,25 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 1
+ Map 2
Map Operator Tree:
TableScan
- alias: y
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ alias: x
+ Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: key is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ predicate: (value is not null and key is not null) (type: boolean)
+ Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
0 _col0 (type: string)
- 1 _col3 (type: string)
- Position of Big Table: 1
+ 1 _col1 (type: string)
+ Position of Big Table: 0
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -136,7 +136,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src
+ base file name: src1
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -146,14 +146,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -166,38 +166,38 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src
- name: default.src
+ name: default.src1
+ name: default.src1
Truncated Path -> Alias:
- /src [y]
+ /src1 [x]
Map 3
Map Operator Tree:
TableScan
- alias: x
- Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ alias: y
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: (value is not null and key is not null) (type: boolean)
- Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
Position of Big Table: 0
Local Work:
Map Reduce Local Work
@@ -206,7 +206,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src1
+ base file name: src
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -216,14 +216,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -236,26 +236,26 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src1
- name: default.src1
+ name: default.src
+ name: default.src
Truncated Path -> Alias:
- /src1 [x]
+ /src [y]
Stage: Stage-1
Spark
#### A masked pattern was here ####
Vertices:
- Map 2
+ Map 1
Map Operator Tree:
TableScan
alias: z
@@ -277,22 +277,22 @@ STAGE PLANS:
1 _col1 (type: string)
outputColumnNames: _col0, _col3
input vertices:
- 1 Map 3
+ 1 Map 2
Position of Big Table: 0
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col1, _col2, _col5
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
input vertices:
- 0 Map 1
- Position of Big Table: 1
+ 1 Map 3
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col5 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col3 (type: string), _col0 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out b/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
index b1efc8f..78bb655 100644
--- a/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/spark/join32_lessSize.q.out
@@ -118,25 +118,25 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 1
+ Map 2
Map Operator Tree:
TableScan
- alias: y
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ alias: x
+ Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: key is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ predicate: (value is not null and key is not null) (type: boolean)
+ Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
0 _col0 (type: string)
- 1 _col3 (type: string)
- Position of Big Table: 1
+ 1 _col1 (type: string)
+ Position of Big Table: 0
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -144,7 +144,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src
+ base file name: src1
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -154,14 +154,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -174,38 +174,38 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src
- name: default.src
+ name: default.src1
+ name: default.src1
Truncated Path -> Alias:
- /src [y]
+ /src1 [x]
Map 3
Map Operator Tree:
TableScan
- alias: x
- Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ alias: y
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: (value is not null and key is not null) (type: boolean)
- Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
Position of Big Table: 0
Local Work:
Map Reduce Local Work
@@ -214,7 +214,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src1
+ base file name: src
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -224,14 +224,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -244,26 +244,26 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src1
- name: default.src1
+ name: default.src
+ name: default.src
Truncated Path -> Alias:
- /src1 [x]
+ /src [y]
Stage: Stage-1
Spark
#### A masked pattern was here ####
Vertices:
- Map 2
+ Map 1
Map Operator Tree:
TableScan
alias: z
@@ -285,22 +285,22 @@ STAGE PLANS:
1 _col1 (type: string)
outputColumnNames: _col0, _col3
input vertices:
- 1 Map 3
+ 1 Map 2
Position of Big Table: 0
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col1, _col2, _col5
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
input vertices:
- 0 Map 1
- Position of Big Table: 1
+ 1 Map 3
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col5 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col3 (type: string), _col0 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -624,7 +624,7 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 3
+ Map 1
Map Operator Tree:
TableScan
alias: x
@@ -699,25 +699,36 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 1
+ Map 2
Map Operator Tree:
TableScan
- alias: w
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ alias: x
+ Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Spark HashTable Sink Operator
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
keys:
0 _col0 (type: string)
- 1 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col3
+ input vertices:
+ 0 Map 1
Position of Big Table: 1
+ Statistics: Num rows: 14 Data size: 108 Basic stats: COMPLETE Column stats: NONE
+ Spark HashTable Sink Operator
+ keys:
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ Position of Big Table: 1
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -725,7 +736,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src
+ base file name: src1
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -735,14 +746,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -755,50 +766,39 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src
- name: default.src
+ name: default.src1
+ name: default.src1
Truncated Path -> Alias:
- /src [w]
+ /src1 [x]
Map 4
Map Operator Tree:
TableScan
- alias: x
- Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ alias: w
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
predicate: key is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Spark HashTable Sink Operator
keys:
0 _col0 (type: string)
1 _col0 (type: string)
- outputColumnNames: _col0, _col1, _col3
- input vertices:
- 0 Map 3
- Position of Big Table: 1
- Statistics: Num rows: 14 Data size: 108 Basic stats: COMPLETE Column stats: NONE
- Spark HashTable Sink Operator
- keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- Position of Big Table: 0
+ Position of Big Table: 0
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -806,7 +806,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src1
+ base file name: src
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -816,14 +816,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -836,26 +836,26 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src1
- name: default.src1
+ name: default.src
+ name: default.src
Truncated Path -> Alias:
- /src1 [x]
+ /src [w]
Stage: Stage-1
Spark
#### A masked pattern was here ####
Vertices:
- Map 2
+ Map 3
Map Operator Tree:
TableScan
alias: w
@@ -873,26 +873,26 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col4
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3
input vertices:
- 1 Map 4
- Position of Big Table: 0
+ 0 Map 2
+ Position of Big Table: 1
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col3, _col6
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
input vertices:
- 0 Map 1
- Position of Big Table: 1
+ 1 Map 4
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col3 (type: string), _col6 (type: string), _col1 (type: string)
+ expressions: _col0 (type: string), _col3 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -1029,7 +1029,7 @@ POSTHOOK: Input: default@src1
POSTHOOK: Output: default@dest_j1
POSTHOOK: Lineage: dest_j1.key EXPRESSION [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)w.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value EXPRESSION [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
PREHOOK: query: select * from dest_j1
PREHOOK: type: QUERY
PREHOOK: Input: default@dest_j1
[23/25] hive git commit: HIVE-11152 : Swapping join inputs in
ASTConverter (Jesus Camacho Rodriguez via John Pullokkaran)
Posted by se...@apache.org.
HIVE-11152 : Swapping join inputs in ASTConverter (Jesus Camacho Rodriguez via John Pullokkaran)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/6eaa32c8
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/6eaa32c8
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/6eaa32c8
Branch: refs/heads/llap
Commit: 6eaa32c85807826577a984f4b65c72d3c76d90db
Parents: b5fb31c
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Wed Jul 1 13:57:24 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Jul 1 13:57:24 2015 -0700
----------------------------------------------------------------------
.../calcite/translator/ASTConverter.java | 20 +-
.../translator/PlanModifierForASTConv.java | 12 +-
.../results/clientpositive/auto_join13.q.out | 26 +-
.../auto_join_without_localtask.q.out | 218 +-
.../clientpositive/cbo_rp_auto_join1.q.out | 57 +-
.../clientpositive/correlationoptimizer6.q.out | 163 +-
ql/src/test/results/clientpositive/join13.q.out | 32 +-
ql/src/test/results/clientpositive/join32.q.out | 36 +-
.../clientpositive/join32_lessSize.q.out | 118 +-
ql/src/test/results/clientpositive/join33.q.out | 36 +-
.../clientpositive/join_alt_syntax.q.out | 104 +-
.../clientpositive/join_cond_pushdown_1.q.out | 42 +-
.../clientpositive/join_cond_pushdown_2.q.out | 62 +-
.../clientpositive/join_cond_pushdown_3.q.out | 42 +-
.../clientpositive/join_cond_pushdown_4.q.out | 62 +-
.../clientpositive/mapjoin_mapjoin.q.out | 120 +-
.../clientpositive/spark/auto_join13.q.out | 40 +-
.../spark/auto_join_without_localtask.q.out | 90 +-
.../results/clientpositive/spark/join13.q.out | 82 +-
.../results/clientpositive/spark/join32.q.out | 102 +-
.../clientpositive/spark/join32_lessSize.q.out | 232 +-
.../results/clientpositive/spark/join33.q.out | 102 +-
.../clientpositive/spark/join_alt_syntax.q.out | 272 +-
.../spark/join_cond_pushdown_1.q.out | 70 +-
.../spark/join_cond_pushdown_2.q.out | 134 +-
.../spark/join_cond_pushdown_3.q.out | 70 +-
.../spark/join_cond_pushdown_4.q.out | 134 +-
.../clientpositive/spark/mapjoin_mapjoin.q.out | 202 +-
.../clientpositive/tez/explainuser_1.q.out | 363 +-
.../clientpositive/tez/explainuser_2.q.out | 3258 +++++++++---------
.../clientpositive/tez/mapjoin_mapjoin.q.out | 266 +-
31 files changed, 3222 insertions(+), 3345 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index 95f43d4..b6995c9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.core.Aggregate.Group;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Filter;
import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
import org.apache.calcite.rel.core.Project;
import org.apache.calcite.rel.core.SemiJoin;
import org.apache.calcite.rel.core.Sort;
@@ -285,9 +286,24 @@ public class ASTConverter {
s = new Schema(left.schema, right.schema);
ASTNode cond = join.getCondition().accept(new RexVisitor(s));
boolean semiJoin = join instanceof SemiJoin;
- ast = ASTBuilder.join(left.ast, right.ast, join.getJoinType(), cond, semiJoin);
- if (semiJoin)
+ if (join.getRight() instanceof Join) {
+ // Invert join inputs; this is done because otherwise the SemanticAnalyzer
+ // methods to merge joins will not kick in
+ JoinRelType type;
+ if (join.getJoinType() == JoinRelType.LEFT) {
+ type = JoinRelType.RIGHT;
+ } else if (join.getJoinType() == JoinRelType.RIGHT) {
+ type = JoinRelType.LEFT;
+ } else {
+ type = join.getJoinType();
+ }
+ ast = ASTBuilder.join(right.ast, left.ast, type, cond, semiJoin);
+ } else {
+ ast = ASTBuilder.join(left.ast, right.ast, join.getJoinType(), cond, semiJoin);
+ }
+ if (semiJoin) {
s = left.schema;
+ }
} else if (r instanceof Union) {
RelNode leftInput = ((Union) r).getInput(0);
RelNode rightInput = ((Union) r).getInput(1);
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
index d8be7ff..5f6be9e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/PlanModifierForASTConv.java
@@ -242,7 +242,15 @@ public class PlanModifierForASTConv {
boolean validParent = true;
if (parent instanceof Join) {
- if (((Join) parent).getRight() == joinNode) {
+ // In Hive AST, right child of join cannot be another join,
+ // thus we need to introduce a project on top of it.
+ // But we only need the additional project if the left child
+ // is another join too; if it is not, ASTConverter will swap
+ // the join inputs, leaving the join operator on the left.
+ // This will help triggering multijoin recognition methods that
+ // are embedded in SemanticAnalyzer.
+ if (((Join) parent).getRight() == joinNode &&
+ (((Join) parent).getLeft() instanceof Join) ) {
validParent = false;
}
} else if (parent instanceof SetOp) {
@@ -255,7 +263,7 @@ public class PlanModifierForASTConv {
private static boolean validFilterParent(RelNode filterNode, RelNode parent) {
boolean validParent = true;
- // TOODO: Verify GB having is not a seperate filter (if so we shouldn't
+ // TODO: Verify GB having is not a separate filter (if so we shouldn't
// introduce derived table)
if (parent instanceof Filter || parent instanceof Join
|| parent instanceof SetOp) {
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/auto_join13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join13.q.out b/ql/src/test/results/clientpositive/auto_join13.q.out
index c5d6b44..952dbf8 100644
--- a/ql/src/test/results/clientpositive/auto_join13.q.out
+++ b/ql/src/test/results/clientpositive/auto_join13.q.out
@@ -21,18 +21,18 @@ JOIN
ON src1.c1 + src2.c3 = src3.c5 AND src3.c5 < 200
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-7 is a root stage
- Stage-2 depends on stages: Stage-7
- Stage-0 depends on stages: Stage-2
+ Stage-8 is a root stage
+ Stage-3 depends on stages: Stage-8
+ Stage-0 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-7
+ Stage: Stage-8
Map Reduce Local Work
Alias -> Map Local Tables:
$hdt$_0:$hdt$_0:src
Fetch Operator
limit: -1
- $hdt$_0:$hdt$_1:$hdt$_1:src
+ $hdt$_0:$hdt$_1:src
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
@@ -49,9 +49,9 @@ STAGE PLANS:
Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
HashTable Sink Operator
keys:
- 0 UDFToDouble(_col0) (type: double)
- 1 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
- $hdt$_0:$hdt$_1:$hdt$_1:src
+ 0 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ 1 UDFToDouble(_col0) (type: double)
+ $hdt$_0:$hdt$_1:src
TableScan
alias: src
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -67,7 +67,7 @@ STAGE PLANS:
0 _col0 (type: string)
1 _col0 (type: string)
- Stage: Stage-2
+ Stage: Stage-3
Map Reduce
Map Operator Tree:
TableScan
@@ -95,12 +95,12 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 UDFToDouble(_col0) (type: double)
- 1 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
- outputColumnNames: _col2, _col3
+ 0 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ 1 UDFToDouble(_col0) (type: double)
+ outputColumnNames: _col1, _col2
Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: hash(_col3,_col2) (type: int)
+ expressions: hash(_col2,_col1) (type: int)
outputColumnNames: _col0
Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
Group By Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/auto_join_without_localtask.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/auto_join_without_localtask.q.out b/ql/src/test/results/clientpositive/auto_join_without_localtask.q.out
index ab77bfe..3d0067b 100644
--- a/ql/src/test/results/clientpositive/auto_join_without_localtask.q.out
+++ b/ql/src/test/results/clientpositive/auto_join_without_localtask.q.out
@@ -654,32 +654,32 @@ POSTHOOK: query: explain
select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 order by a.key, a.value limit 40
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-10 is a root stage , consists of Stage-13, Stage-14, Stage-3
- Stage-13 has a backup stage: Stage-3
- Stage-8 depends on stages: Stage-13
- Stage-7 depends on stages: Stage-3, Stage-8, Stage-9 , consists of Stage-5, Stage-12, Stage-1
- Stage-5 has a backup stage: Stage-1
- Stage-2 depends on stages: Stage-1, Stage-5, Stage-6
- Stage-12 has a backup stage: Stage-1
+ Stage-11 is a root stage , consists of Stage-14, Stage-15, Stage-1
+ Stage-14 has a backup stage: Stage-1
+ Stage-9 depends on stages: Stage-14
+ Stage-8 depends on stages: Stage-1, Stage-9, Stage-10 , consists of Stage-12, Stage-7, Stage-2
+ Stage-12 has a backup stage: Stage-2
Stage-6 depends on stages: Stage-12
+ Stage-3 depends on stages: Stage-2, Stage-6, Stage-7
+ Stage-7 has a backup stage: Stage-2
+ Stage-2
+ Stage-15 has a backup stage: Stage-1
+ Stage-10 depends on stages: Stage-15
Stage-1
- Stage-14 has a backup stage: Stage-3
- Stage-9 depends on stages: Stage-14
- Stage-3
- Stage-0 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-10
+ Stage: Stage-11
Conditional Operator
- Stage: Stage-13
+ Stage: Stage-14
Map Reduce Local Work
Alias -> Map Local Tables:
- $hdt$_1:$hdt$_2:a
+ $hdt$_2:a
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
- $hdt$_1:$hdt$_2:a
+ $hdt$_2:a
TableScan
alias: a
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -695,7 +695,7 @@ STAGE PLANS:
0 _col0 (type: string)
1 _col0 (type: string)
- Stage: Stage-8
+ Stage: Stage-9
Map Reduce
Map Operator Tree:
TableScan
@@ -725,12 +725,17 @@ STAGE PLANS:
Local Work:
Map Reduce Local Work
- Stage: Stage-7
+ Stage: Stage-8
Conditional Operator
- Stage: Stage-5
- Map Reduce
- Map Operator Tree:
+ Stage: Stage-12
+ Map Reduce Local Work
+ Alias -> Map Local Tables:
+ $hdt$_0:a
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ $hdt$_0:a
TableScan
alias: a
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -741,35 +746,33 @@ STAGE PLANS:
expressions: value (type: string)
outputColumnNames: _col0
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Map Join Operator
- condition map:
- Inner Join 0 to 1
+ HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col2
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string), _col2 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+
+ Stage: Stage-6
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
Local Work:
Map Reduce Local Work
- Alias -> Map Local Tables:
- $INTNAME
- Fetch Operator
- limit: -1
- Alias -> Map Local Operator Tree:
- $INTNAME
- TableScan
- Stage: Stage-2
+ Stage: Stage-3
Map Reduce
Map Operator Tree:
TableScan
@@ -793,14 +796,9 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Stage: Stage-12
- Map Reduce Local Work
- Alias -> Map Local Tables:
- $hdt$_0:a
- Fetch Operator
- limit: -1
- Alias -> Map Local Operator Tree:
- $hdt$_0:a
+ Stage: Stage-7
+ Map Reduce
+ Map Operator Tree:
TableScan
alias: a
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -811,40 +809,41 @@ STAGE PLANS:
expressions: value (type: string)
outputColumnNames: _col0
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- HashTable Sink Operator
+ Map Join Operator
+ condition map:
+ Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
-
- Stage: Stage-6
- Map Reduce
- Map Operator Tree:
- TableScan
- Map Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col2
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string), _col2 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
Local Work:
Map Reduce Local Work
+ Alias -> Map Local Tables:
+ $INTNAME
+ Fetch Operator
+ limit: -1
+ Alias -> Map Local Operator Tree:
+ $INTNAME
+ TableScan
- Stage: Stage-1
+ Stage: Stage-2
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: string)
+ TableScan
alias: a
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -859,41 +858,30 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- TableScan
- Reduce Output Operator
- key expressions: _col1 (type: string)
- sort order: +
- Map-reduce partition columns: _col1 (type: string)
- Statistics: Num rows: 182 Data size: 1939 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col2
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string), _col2 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-14
+ Stage: Stage-15
Map Reduce Local Work
Alias -> Map Local Tables:
- $hdt$_1:$hdt$_1:a
+ $hdt$_1:a
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
- $hdt$_1:$hdt$_1:a
+ $hdt$_1:a
TableScan
alias: a
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -909,7 +897,7 @@ STAGE PLANS:
0 _col0 (type: string)
1 _col0 (type: string)
- Stage: Stage-9
+ Stage: Stage-10
Map Reduce
Map Operator Tree:
TableScan
@@ -939,7 +927,7 @@ STAGE PLANS:
Local Work:
Map Reduce Local Work
- Stage: Stage-3
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -1003,13 +991,13 @@ POSTHOOK: query: select a.* from src a join src b on a.key=b.key join src c on a
POSTHOOK: type: QUERY
POSTHOOK: Input: default@src
#### A masked pattern was here ####
-RUN: Stage-10:CONDITIONAL
-RUN: Stage-13:MAPREDLOCAL
-RUN: Stage-8:MAPRED
-RUN: Stage-7:CONDITIONAL
+RUN: Stage-11:CONDITIONAL
+RUN: Stage-14:MAPREDLOCAL
+RUN: Stage-9:MAPRED
+RUN: Stage-8:CONDITIONAL
RUN: Stage-12:MAPREDLOCAL
RUN: Stage-6:MAPRED
-RUN: Stage-2:MAPRED
+RUN: Stage-3:MAPRED
103 val_103
103 val_103
103 val_103
@@ -1064,13 +1052,13 @@ select a.* from src a join src b on a.key=b.key join src c on a.value=c.value wh
POSTHOOK: type: QUERY
POSTHOOK: Input: default@src
#### A masked pattern was here ####
-RUN: Stage-10:CONDITIONAL
-RUN: Stage-13:MAPREDLOCAL
-RUN: Stage-3:MAPRED
-RUN: Stage-7:CONDITIONAL
-RUN: Stage-12:MAPREDLOCAL
+RUN: Stage-11:CONDITIONAL
+RUN: Stage-14:MAPREDLOCAL
RUN: Stage-1:MAPRED
+RUN: Stage-8:CONDITIONAL
+RUN: Stage-12:MAPREDLOCAL
RUN: Stage-2:MAPRED
+RUN: Stage-3:MAPRED
103 val_103
103 val_103
103 val_103
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
index f4b25ae..79b4650 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
@@ -1127,8 +1127,7 @@ select count(*) from
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-0 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-1
STAGE PLANS:
Stage: Stage-1
@@ -1144,57 +1143,21 @@ STAGE PLANS:
expressions: key (type: int)
outputColumnNames: _col0
Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
- TableScan
- alias: a
- Statistics: Num rows: 10 Data size: 70 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator
- predicate: (key < 6) (type: boolean)
- Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
- Select Operator
- expressions: key (type: int)
- outputColumnNames: _col0
- Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
Sorted Merge Bucket Map Join Operator
condition map:
Inner Join 0 to 1
+ Inner Join 0 to 2
keys:
0 _col0 (type: int)
1 _col0 (type: int)
- outputColumnNames: _col0
- Reduce Output Operator
- key expressions: _col0 (type: int)
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Reduce Operator Tree:
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: int)
- 1 _col0 (type: int)
- Group By Operator
- aggregations: count()
- mode: hash
- outputColumnNames: _col0
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
-
- Stage: Stage-2
- Map Reduce
- Map Operator Tree:
- TableScan
- Reduce Output Operator
- sort order:
- value expressions: _col0 (type: bigint)
+ 2 _col0 (type: int)
+ Group By Operator
+ aggregations: count()
+ mode: hash
+ outputColumnNames: _col0
+ Reduce Output Operator
+ sort order:
+ value expressions: _col0 (type: bigint)
Reduce Operator Tree:
Group By Operator
aggregations: count(VALUE._col0)
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/correlationoptimizer6.q.out b/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
index 38e51fa..85e447c 100644
--- a/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
+++ b/ql/src/test/results/clientpositive/correlationoptimizer6.q.out
@@ -1787,14 +1787,13 @@ JOIN
ON zz.key=yy.key
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-1 is a root stage
- Stage-2 depends on stages: Stage-1
- Stage-3 depends on stages: Stage-2, Stage-5
- Stage-5 is a root stage
- Stage-0 depends on stages: Stage-3
+ Stage-3 is a root stage
+ Stage-4 depends on stages: Stage-3
+ Stage-1 depends on stages: Stage-4
+ Stage-0 depends on stages: Stage-1
STAGE PLANS:
- Stage: Stage-1
+ Stage: Stage-3
Map Reduce
Map Operator Tree:
TableScan
@@ -1849,7 +1848,7 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-2
+ Stage: Stage-4
Map Reduce
Map Operator Tree:
TableScan
@@ -1873,45 +1872,7 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-3
- Map Reduce
- Map Operator Tree:
- TableScan
- Reduce Output Operator
- key expressions: _col0 (type: string)
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: bigint)
- TableScan
- Reduce Output Operator
- key expressions: _col0 (type: string)
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string)
- Reduce Operator Tree:
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
- outputColumnNames: _col0, _col1, _col3
- Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col3 (type: string), _col0 (type: string), _col1 (type: bigint)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
- Stage: Stage-5
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -1944,21 +1905,35 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 137 Data size: 1455 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: bigint)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
+ Inner Join 0 to 2
keys:
0 _col0 (type: string)
1 _col0 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ 2 _col0 (type: string)
+ outputColumnNames: _col1, _col2, _col3
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: string), _col2 (type: string), _col3 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
@@ -2069,20 +2044,20 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
TableScan
- alias: zz
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ alias: xx
+ Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: string)
sort order: +
Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
TableScan
alias: zz
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -2099,23 +2074,46 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
TableScan
- alias: xx
- Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ alias: zz
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: key is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
key expressions: _col0 (type: string)
sort order: +
Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Reduce Operator Tree:
Demux Operator
Statistics: Num rows: 763 Data size: 8067 Basic stats: COMPLETE Column stats: NONE
+ Mux Operator
+ Statistics: Num rows: 1527 Data size: 16134 Basic stats: COMPLETE Column stats: NONE
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ Inner Join 0 to 2
+ keys:
+ 0 _col0 (type: string)
+ 1 _col0 (type: string)
+ 2 _col0 (type: string)
+ outputColumnNames: _col1, _col2, _col3
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: string), _col2 (type: string), _col3 (type: bigint)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Join Operator
condition map:
Inner Join 0 to 1
@@ -2133,17 +2131,19 @@ STAGE PLANS:
outputColumnNames: _col0, _col1
Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
Mux Operator
- Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Statistics: Num rows: 1527 Data size: 16134 Basic stats: COMPLETE Column stats: NONE
Join Operator
condition map:
Inner Join 0 to 1
+ Inner Join 0 to 2
keys:
0 _col0 (type: string)
1 _col0 (type: string)
- outputColumnNames: _col0, _col1, _col3
+ 2 _col0 (type: string)
+ outputColumnNames: _col1, _col2, _col3
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
Select Operator
- expressions: _col3 (type: string), _col0 (type: string), _col1 (type: bigint)
+ expressions: _col1 (type: string), _col2 (type: string), _col3 (type: bigint)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
File Output Operator
@@ -2153,35 +2153,6 @@ STAGE PLANS:
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
- outputColumnNames: _col0, _col1
- Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- Mux Operator
- Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col0 (type: string)
- 1 _col0 (type: string)
- outputColumnNames: _col0, _col1, _col3
- Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- Select Operator
- expressions: _col3 (type: string), _col0 (type: string), _col1 (type: bigint)
- outputColumnNames: _col0, _col1, _col2
- Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 0 Data size: 0 Basic stats: NONE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join13.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join13.q.out b/ql/src/test/results/clientpositive/join13.q.out
index 3f3f7e5..3b921b9 100644
--- a/ql/src/test/results/clientpositive/join13.q.out
+++ b/ql/src/test/results/clientpositive/join13.q.out
@@ -25,12 +25,12 @@ JOIN
ON src1.c1 + src2.c3 = src3.c5 AND src3.c5 < 200
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-2 is a root stage
- Stage-1 depends on stages: Stage-2
- Stage-0 depends on stages: Stage-1
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
STAGE PLANS:
- Stage: Stage-2
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -83,10 +83,17 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-1
+ Stage: Stage-2
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ sort order: +
+ Map-reduce partition columns: (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string), _col2 (type: string)
+ TableScan
alias: src
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -101,24 +108,17 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: UDFToDouble(_col0) (type: double)
Statistics: Num rows: 83 Data size: 881 Basic stats: COMPLETE Column stats: NONE
- TableScan
- Reduce Output Operator
- key expressions: (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
- sort order: +
- Map-reduce partition columns: (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
- Statistics: Num rows: 91 Data size: 969 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 UDFToDouble(_col0) (type: double)
- 1 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
- outputColumnNames: _col2, _col3
+ 0 (UDFToDouble(_col2) + UDFToDouble(_col0)) (type: double)
+ 1 UDFToDouble(_col0) (type: double)
+ outputColumnNames: _col1, _col2
Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col3 (type: string), _col2 (type: string)
+ expressions: _col2 (type: string), _col1 (type: string)
outputColumnNames: _col0, _col1
Statistics: Num rows: 100 Data size: 1065 Basic stats: COMPLETE Column stats: NONE
File Output Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join32.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join32.q.out b/ql/src/test/results/clientpositive/join32.q.out
index 49e302a..5795669 100644
--- a/ql/src/test/results/clientpositive/join32.q.out
+++ b/ql/src/test/results/clientpositive/join32.q.out
@@ -100,19 +100,19 @@ TOK_QUERY
STAGE DEPENDENCIES:
- Stage-7 is a root stage
- Stage-5 depends on stages: Stage-7
- Stage-0 depends on stages: Stage-5
- Stage-2 depends on stages: Stage-0
+ Stage-8 is a root stage
+ Stage-6 depends on stages: Stage-8
+ Stage-0 depends on stages: Stage-6
+ Stage-3 depends on stages: Stage-0
STAGE PLANS:
- Stage: Stage-7
+ Stage: Stage-8
Map Reduce Local Work
Alias -> Map Local Tables:
$hdt$_0:y
Fetch Operator
limit: -1
- $hdt$_1:$hdt$_2:x
+ $hdt$_2:x
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
@@ -131,10 +131,10 @@ STAGE PLANS:
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- Position of Big Table: 1
- $hdt$_1:$hdt$_2:x
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ Position of Big Table: 0
+ $hdt$_2:x
TableScan
alias: x
Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -153,7 +153,7 @@ STAGE PLANS:
1 _col1 (type: string)
Position of Big Table: 0
- Stage: Stage-5
+ Stage: Stage-6
Map Reduce
Map Operator Tree:
TableScan
@@ -181,13 +181,13 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col1, _col2, _col5
- Position of Big Table: 1
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col5 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col3 (type: string), _col0 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -356,7 +356,7 @@ STAGE PLANS:
name: default.srcpart
name: default.srcpart
Truncated Path -> Alias:
- /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:$hdt$_1:z]
+ /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:z]
Stage: Stage-0
Move Operator
@@ -380,7 +380,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: default.dest_j1
- Stage: Stage-2
+ Stage: Stage-3
Stats-Aggr Operator
#### A masked pattern was here ####
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join32_lessSize.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join32_lessSize.q.out b/ql/src/test/results/clientpositive/join32_lessSize.q.out
index 736a912..c027dba 100644
--- a/ql/src/test/results/clientpositive/join32_lessSize.q.out
+++ b/ql/src/test/results/clientpositive/join32_lessSize.q.out
@@ -108,22 +108,22 @@ TOK_QUERY
STAGE DEPENDENCIES:
- Stage-8 is a root stage
+ Stage-9 is a root stage
+ Stage-7 depends on stages: Stage-9
+ Stage-8 depends on stages: Stage-7
Stage-6 depends on stages: Stage-8
- Stage-7 depends on stages: Stage-6
- Stage-5 depends on stages: Stage-7
- Stage-0 depends on stages: Stage-5
- Stage-2 depends on stages: Stage-0
+ Stage-0 depends on stages: Stage-6
+ Stage-3 depends on stages: Stage-0
STAGE PLANS:
- Stage: Stage-8
+ Stage: Stage-9
Map Reduce Local Work
Alias -> Map Local Tables:
- $hdt$_1:$hdt$_2:x
+ $hdt$_2:x
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
- $hdt$_1:$hdt$_2:x
+ $hdt$_2:x
TableScan
alias: x
Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -142,7 +142,7 @@ STAGE PLANS:
1 _col1 (type: string)
Position of Big Table: 0
- Stage: Stage-6
+ Stage: Stage-7
Map Reduce
Map Operator Tree:
TableScan
@@ -279,9 +279,9 @@ STAGE PLANS:
name: default.srcpart
name: default.srcpart
Truncated Path -> Alias:
- /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:$hdt$_1:z]
+ /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:z]
- Stage: Stage-7
+ Stage: Stage-8
Map Reduce Local Work
Alias -> Map Local Tables:
$hdt$_0:y
@@ -303,11 +303,11 @@ STAGE PLANS:
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- Position of Big Table: 1
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ Position of Big Table: 0
- Stage: Stage-5
+ Stage: Stage-6
Map Reduce
Map Operator Tree:
TableScan
@@ -316,13 +316,13 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col1, _col2, _col5
- Position of Big Table: 1
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col5 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col3 (type: string), _col0 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -445,7 +445,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: default.dest_j1
- Stage: Stage-2
+ Stage: Stage-3
Stats-Aggr Operator
#### A masked pattern was here ####
@@ -653,24 +653,24 @@ TOK_QUERY
STAGE DEPENDENCIES:
- Stage-11 is a root stage
+ Stage-13 is a root stage
+ Stage-10 depends on stages: Stage-13
+ Stage-12 depends on stages: Stage-10
+ Stage-9 depends on stages: Stage-12
+ Stage-11 depends on stages: Stage-9
Stage-8 depends on stages: Stage-11
- Stage-10 depends on stages: Stage-8
- Stage-7 depends on stages: Stage-10
- Stage-9 depends on stages: Stage-7
- Stage-6 depends on stages: Stage-9
- Stage-0 depends on stages: Stage-6
- Stage-2 depends on stages: Stage-0
+ Stage-0 depends on stages: Stage-8
+ Stage-4 depends on stages: Stage-0
STAGE PLANS:
- Stage: Stage-11
+ Stage: Stage-13
Map Reduce Local Work
Alias -> Map Local Tables:
- $hdt$_1:$hdt$_2:$hdt$_2:x
+ $hdt$_2:x
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
- $hdt$_1:$hdt$_2:$hdt$_2:x
+ $hdt$_2:x
TableScan
alias: x
Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -689,7 +689,7 @@ STAGE PLANS:
1 _col0 (type: string)
Position of Big Table: 1
- Stage: Stage-8
+ Stage: Stage-10
Map Reduce
Map Operator Tree:
TableScan
@@ -780,16 +780,16 @@ STAGE PLANS:
name: default.src1
name: default.src1
Truncated Path -> Alias:
- /src1 [$hdt$_1:$hdt$_2:$hdt$_3:x]
+ /src1 [$hdt$_3:x]
- Stage: Stage-10
+ Stage: Stage-12
Map Reduce Local Work
Alias -> Map Local Tables:
- $hdt$_1:$hdt$_1:w
+ $hdt$_1:w
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
- $hdt$_1:$hdt$_1:w
+ $hdt$_1:w
TableScan
alias: w
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -804,11 +804,11 @@ STAGE PLANS:
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- Position of Big Table: 1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ Position of Big Table: 0
- Stage: Stage-7
+ Stage: Stage-9
Map Reduce
Map Operator Tree:
TableScan
@@ -817,10 +817,10 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col4
- Position of Big Table: 1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3
+ Position of Big Table: 0
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -831,7 +831,7 @@ STAGE PLANS:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
properties:
- columns _col1,_col4
+ columns _col0,_col3
columns.types string,string
escape.delim \
serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
@@ -846,7 +846,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: -mr-10002
+ base file name: -mr-10001
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
properties:
@@ -911,7 +911,7 @@ STAGE PLANS:
Truncated Path -> Alias:
#### A masked pattern was here ####
- Stage: Stage-9
+ Stage: Stage-11
Map Reduce Local Work
Alias -> Map Local Tables:
$hdt$_0:w
@@ -934,10 +934,10 @@ STAGE PLANS:
HashTable Sink Operator
keys:
0 _col0 (type: string)
- 1 _col1 (type: string)
- Position of Big Table: 1
+ 1 _col0 (type: string)
+ Position of Big Table: 0
- Stage: Stage-6
+ Stage: Stage-8
Map Reduce
Map Operator Tree:
TableScan
@@ -947,12 +947,12 @@ STAGE PLANS:
Inner Join 0 to 1
keys:
0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1, _col3, _col6
- Position of Big Table: 1
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col3 (type: string), _col6 (type: string), _col1 (type: string)
+ expressions: _col0 (type: string), _col3 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -993,11 +993,11 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: -mr-10001
+ base file name: -mr-10002
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
properties:
- columns _col1,_col4
+ columns _col0,_col3
columns.types string,string
escape.delim \
serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
@@ -1006,7 +1006,7 @@ STAGE PLANS:
input format: org.apache.hadoop.mapred.SequenceFileInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
properties:
- columns _col1,_col4
+ columns _col0,_col3
columns.types string,string
escape.delim \
serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
@@ -1085,7 +1085,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: default.dest_j1
- Stage: Stage-2
+ Stage: Stage-4
Stats-Aggr Operator
#### A masked pattern was here ####
@@ -1109,7 +1109,7 @@ POSTHOOK: Input: default@src1
POSTHOOK: Output: default@dest_j1
POSTHOOK: Lineage: dest_j1.key EXPRESSION [(src1)x.FieldSchema(name:key, type:string, comment:default), ]
POSTHOOK: Lineage: dest_j1.val2 SIMPLE [(src)w.FieldSchema(name:value, type:string, comment:default), ]
-POSTHOOK: Lineage: dest_j1.value SIMPLE [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: dest_j1.value EXPRESSION [(src1)x.FieldSchema(name:value, type:string, comment:default), ]
PREHOOK: query: select * from dest_j1
PREHOOK: type: QUERY
PREHOOK: Input: default@dest_j1
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join33.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join33.q.out b/ql/src/test/results/clientpositive/join33.q.out
index 49e302a..5795669 100644
--- a/ql/src/test/results/clientpositive/join33.q.out
+++ b/ql/src/test/results/clientpositive/join33.q.out
@@ -100,19 +100,19 @@ TOK_QUERY
STAGE DEPENDENCIES:
- Stage-7 is a root stage
- Stage-5 depends on stages: Stage-7
- Stage-0 depends on stages: Stage-5
- Stage-2 depends on stages: Stage-0
+ Stage-8 is a root stage
+ Stage-6 depends on stages: Stage-8
+ Stage-0 depends on stages: Stage-6
+ Stage-3 depends on stages: Stage-0
STAGE PLANS:
- Stage: Stage-7
+ Stage: Stage-8
Map Reduce Local Work
Alias -> Map Local Tables:
$hdt$_0:y
Fetch Operator
limit: -1
- $hdt$_1:$hdt$_2:x
+ $hdt$_2:x
Fetch Operator
limit: -1
Alias -> Map Local Operator Tree:
@@ -131,10 +131,10 @@ STAGE PLANS:
Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- Position of Big Table: 1
- $hdt$_1:$hdt$_2:x
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ Position of Big Table: 0
+ $hdt$_2:x
TableScan
alias: x
Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -153,7 +153,7 @@ STAGE PLANS:
1 _col1 (type: string)
Position of Big Table: 0
- Stage: Stage-5
+ Stage: Stage-6
Map Reduce
Map Operator Tree:
TableScan
@@ -181,13 +181,13 @@ STAGE PLANS:
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col1, _col2, _col5
- Position of Big Table: 1
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col5 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col3 (type: string), _col0 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -356,7 +356,7 @@ STAGE PLANS:
name: default.srcpart
name: default.srcpart
Truncated Path -> Alias:
- /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:$hdt$_1:z]
+ /srcpart/ds=2008-04-08/hr=11 [$hdt$_1:z]
Stage: Stage-0
Move Operator
@@ -380,7 +380,7 @@ STAGE PLANS:
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
name: default.dest_j1
- Stage: Stage-2
+ Stage: Stage-3
Stats-Aggr Operator
#### A masked pattern was here ####
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join_alt_syntax.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_alt_syntax.q.out b/ql/src/test/results/clientpositive/join_alt_syntax.q.out
index cc908c1..d119ab5 100644
--- a/ql/src/test/results/clientpositive/join_alt_syntax.q.out
+++ b/ql/src/test/results/clientpositive/join_alt_syntax.q.out
@@ -359,13 +359,13 @@ where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey
and p1.p_partkey = p2.p_partkey
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-3 is a root stage
- Stage-2 depends on stages: Stage-3
- Stage-1 depends on stages: Stage-2
- Stage-0 depends on stages: Stage-1
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-3
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -418,6 +418,13 @@ STAGE PLANS:
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col3 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col3 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -432,21 +439,14 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- TableScan
- Reduce Output Operator
- key expressions: _col3 (type: string)
- sort order: +
- Map-reduce partition columns: _col3 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col4
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -455,10 +455,17 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-1
+ Stage: Stage-3
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -474,24 +481,17 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: string)
- TableScan
- Reduce Output Operator
- key expressions: _col1 (type: int)
- sort order: +
- Map-reduce partition columns: _col1 (type: int)
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string), _col2 (type: string), _col4 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col1 (type: int)
- outputColumnNames: _col1, _col2, _col4, _col6
+ 1 _col0 (type: int)
+ outputColumnNames: _col1, _col3, _col4, _col6
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: string), _col6 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -519,13 +519,13 @@ where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey
and p1.p_partkey = p2.p_partkey
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-3 is a root stage
- Stage-2 depends on stages: Stage-3
- Stage-1 depends on stages: Stage-2
- Stage-0 depends on stages: Stage-1
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-3
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -578,6 +578,13 @@ STAGE PLANS:
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col3 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col3 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -592,21 +599,14 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- TableScan
- Reduce Output Operator
- key expressions: _col3 (type: string)
- sort order: +
- Map-reduce partition columns: _col3 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col4
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
File Output Operator
compressed: false
@@ -615,10 +615,17 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-1
+ Stage: Stage-3
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -634,24 +641,17 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: string)
- TableScan
- Reduce Output Operator
- key expressions: _col1 (type: int)
- sort order: +
- Map-reduce partition columns: _col1 (type: int)
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string), _col2 (type: string), _col4 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col1 (type: int)
- outputColumnNames: _col1, _col2, _col4, _col6
+ 1 _col0 (type: int)
+ outputColumnNames: _col1, _col3, _col4, _col6
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col4 (type: string), _col6 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
File Output Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out b/ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out
index b6e5b50..d565c7f 100644
--- a/ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out
+++ b/ql/src/test/results/clientpositive/join_cond_pushdown_1.q.out
@@ -281,7 +281,7 @@ STAGE PLANS:
Processor Tree:
ListSink
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Stage-1:MAPRED' is a cross product
+Warning: Shuffle Join JOIN[15][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Stage 'Stage-2:MAPRED' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3 on p2.p_partkey = 1 and p3.p_name = p2.p_name
PREHOOK: type: QUERY
@@ -289,12 +289,12 @@ POSTHOOK: query: explain select *
from part p1 join part p2 join part p3 on p2.p_partkey = 1 and p3.p_name = p2.p_name
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-2 is a root stage
- Stage-1 depends on stages: Stage-2
- Stage-0 depends on stages: Stage-1
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
STAGE PLANS:
- Stage: Stage-2
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -338,21 +338,22 @@ STAGE PLANS:
1 _col1 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- table:
- input format: org.apache.hadoop.mapred.SequenceFileInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-1
+ Stage: Stage-2
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Select Operator
@@ -363,11 +364,6 @@ STAGE PLANS:
sort order:
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- TableScan
- Reduce Output Operator
- sort order:
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
@@ -375,10 +371,10 @@ STAGE PLANS:
keys:
0
1
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), 1 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), 1 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
File Output Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/join_cond_pushdown_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_cond_pushdown_2.q.out b/ql/src/test/results/clientpositive/join_cond_pushdown_2.q.out
index 98008ad..55c37e4 100644
--- a/ql/src/test/results/clientpositive/join_cond_pushdown_2.q.out
+++ b/ql/src/test/results/clientpositive/join_cond_pushdown_2.q.out
@@ -141,13 +141,13 @@ from part p1 join part p2 join part p3 on p2.p_name = p1.p_name join part p4 on
and p1.p_partkey = p2.p_partkey
POSTHOOK: type: QUERY
STAGE DEPENDENCIES:
- Stage-3 is a root stage
- Stage-2 depends on stages: Stage-3
- Stage-1 depends on stages: Stage-2
- Stage-0 depends on stages: Stage-1
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-3 depends on stages: Stage-2
+ Stage-0 depends on stages: Stage-3
STAGE PLANS:
- Stage: Stage-3
+ Stage: Stage-1
Map Reduce
Map Operator Tree:
TableScan
@@ -202,6 +202,13 @@ STAGE PLANS:
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col10 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col10 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -217,20 +224,13 @@ STAGE PLANS:
Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- TableScan
- Reduce Output Operator
- key expressions: _col10 (type: string)
- sort order: +
- Map-reduce partition columns: _col10 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col1 (type: string)
- 1 _col10 (type: string)
+ 0 _col10 (type: string)
+ 1 _col1 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -240,10 +240,17 @@ STAGE PLANS:
output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
- Stage: Stage-1
+ Stage: Stage-3
Map Reduce
Map Operator Tree:
TableScan
+ Reduce Output Operator
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
+ TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -259,33 +266,22 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- TableScan
- Reduce Output Operator
- key expressions: _col9 (type: int)
- sort order: +
- Map-reduce partition columns: _col9 (type: int)
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
0 _col0 (type: int)
- 1 _col9 (type: int)
+ 1 _col0 (type: int)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col27 (type: int), _col28 (type: string), _col29 (type: string), _col30 (type: string), _col31 (type: string), _col32 (type: int), _col33 (type: string), _col34 (type: double), _col35 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
+ File Output Operator
+ compressed: false
Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
[18/25] hive git commit: HIVE-11152 : Swapping join inputs in
ASTConverter (Jesus Camacho Rodriguez via John Pullokkaran)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
index f2610de..23df010 100644
--- a/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/tez/mapjoin_mapjoin.q.out
@@ -63,82 +63,12 @@ STAGE PLANS:
Stage: Stage-1
Tez
Edges:
- Map 2 <- Map 1 (BROADCAST_EDGE), Map 3 (BROADCAST_EDGE)
+ Map 1 <- Map 2 (BROADCAST_EDGE), Map 3 (BROADCAST_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
Map Operator Tree:
TableScan
- alias: src
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- GatherStats: false
- Filter Operator
- isSamplingPred: false
- predicate: value is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: value (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- tag: 0
- auto parallelism: true
- Path -> Alias:
-#### A masked pattern was here ####
- Path -> Partition:
-#### A masked pattern was here ####
- Partition
- base file name: src
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- properties:
- COLUMN_STATS_ACCURATE true
- bucket_count -1
- columns key,value
- columns.comments 'default','default'
- columns.types string:string
-#### A masked pattern was here ####
- name default.src
- numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
- serialization.format 1
- serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
-#### A masked pattern was here ####
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- properties:
- COLUMN_STATS_ACCURATE true
- bucket_count -1
- columns key,value
- columns.comments 'default','default'
- columns.types string:string
-#### A masked pattern was here ####
- name default.src
- numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
- serialization.format 1
- serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
-#### A masked pattern was here ####
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src
- name: default.src
- Truncated Path -> Alias:
- /src [src]
- Map 2
- Map Operator Tree:
- TableScan
alias: srcpart
Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
@@ -153,54 +83,50 @@ STAGE PLANS:
Map Join Operator
condition map:
Inner Join 0 to 1
- Estimated key counts: Map 3 => 13
+ Estimated key counts: Map 2 => 13
keys:
0 _col0 (type: string)
1 _col0 (type: string)
outputColumnNames: _col0, _col1
input vertices:
- 1 Map 3
+ 1 Map 2
Position of Big Table: 0
Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
HybridGraceHashJoin: true
Map Join Operator
condition map:
Inner Join 0 to 1
- Estimated key counts: Map 1 => 250
+ Estimated key counts: Map 3 => 250
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0
input vertices:
- 0 Map 1
- Position of Big Table: 1
+ 1 Map 3
+ Position of Big Table: 0
Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
HybridGraceHashJoin: true
- Select Operator
- expressions: _col1 (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- GlobalTableId: 0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
#### A masked pattern was here ####
- NumFilesPerFileSink: 1
- Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
#### A masked pattern was here ####
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- properties:
- columns _col0
- columns.types string
- escape.delim \
- hive.serialization.extend.additional.nesting.levels true
- serialization.format 1
- serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- TotalFiles: 1
- GatherStats: false
- MultiFileSpray: false
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ columns _col0
+ columns.types string
+ escape.delim \
+ hive.serialization.extend.additional.nesting.levels true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
Path -> Alias:
#### A masked pattern was here ####
Path -> Partition:
@@ -393,7 +319,7 @@ STAGE PLANS:
/srcpart/ds=2008-04-08/hr=12 [srcpart]
/srcpart/ds=2008-04-09/hr=11 [srcpart]
/srcpart/ds=2008-04-09/hr=12 [srcpart]
- Map 3
+ Map 2
Map Operator Tree:
TableScan
alias: src1
@@ -463,6 +389,76 @@ STAGE PLANS:
name: default.src1
Truncated Path -> Alias:
/src1 [src1]
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: value (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ tag: 1
+ auto parallelism: true
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: src
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ bucket_count -1
+ columns key,value
+ columns.comments 'default','default'
+ columns.types string:string
+#### A masked pattern was here ####
+ name default.src
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 5812
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE true
+ bucket_count -1
+ columns key,value
+ columns.comments 'default','default'
+ columns.types string:string
+#### A masked pattern was here ####
+ name default.src
+ numFiles 1
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ totalSize 5812
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ name: default.src
+ name: default.src
+ Truncated Path -> Alias:
+ /src [src]
Stage: Stage-0
Fetch Operator
@@ -484,29 +480,12 @@ STAGE PLANS:
Stage: Stage-1
Tez
Edges:
- Map 2 <- Map 1 (BROADCAST_EDGE), Map 3 (BROADCAST_EDGE)
+ Map 1 <- Map 2 (BROADCAST_EDGE), Map 3 (BROADCAST_EDGE)
#### A masked pattern was here ####
Vertices:
Map 1
Map Operator Tree:
TableScan
- alias: src
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: (value > 'val_450') (type: boolean)
- Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: value (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: string)
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
- Map 2
- Map Operator Tree:
- TableScan
alias: srcpart
Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
Filter Operator
@@ -524,32 +503,28 @@ STAGE PLANS:
1 _col0 (type: string)
outputColumnNames: _col0, _col1
input vertices:
- 1 Map 3
+ 1 Map 2
Statistics: Num rows: 183 Data size: 1951 Basic stats: COMPLETE Column stats: NONE
HybridGraceHashJoin: true
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0
input vertices:
- 0 Map 1
+ 1 Map 3
Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
HybridGraceHashJoin: true
- Select Operator
- expressions: _col1 (type: string)
- outputColumnNames: _col0
+ File Output Operator
+ compressed: false
Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Map 3
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ Map 2
Map Operator Tree:
TableScan
alias: src1
@@ -566,6 +541,23 @@ STAGE PLANS:
sort order: +
Map-reduce partition columns: _col0 (type: string)
Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Map 3
+ Map Operator Tree:
+ TableScan
+ alias: src
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: (value > 'val_450') (type: boolean)
+ Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: value (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
Stage: Stage-0
Fetch Operator
[13/25] hive git commit: HIVE-11055 HPL/SQL - Implementing Procedural
SQL in Hive (PL/HQL Contribution) (Dmitry Tolpeko via gates)
Posted by se...@apache.org.
HIVE-11055 HPL/SQL - Implementing Procedural SQL in Hive (PL/HQL Contribution) (Dmitry Tolpeko via gates)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/052643cb
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/052643cb
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/052643cb
Branch: refs/heads/llap
Commit: 052643cb8d1fa3811cbc0e1f56b0cd10ca419b8d
Parents: edb7b88
Author: Alan Gates <ga...@hortonworks.com>
Authored: Wed Jul 1 09:16:44 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Wed Jul 1 09:16:44 2015 -0700
----------------------------------------------------------------------
bin/ext/hplsql.sh | 37 +
bin/hplsql | 25 +
bin/hplsql.cmd | 58 +
hplsql/pom.xml | 123 ++
.../antlr4/org/apache/hive/hplsql/Hplsql.g4 | 1426 +++++++++++++
.../java/org/apache/hive/hplsql/Arguments.java | 206 ++
.../main/java/org/apache/hive/hplsql/Conf.java | 175 ++
.../main/java/org/apache/hive/hplsql/Conn.java | 243 +++
.../java/org/apache/hive/hplsql/Converter.java | 56 +
.../main/java/org/apache/hive/hplsql/Copy.java | 426 ++++
.../main/java/org/apache/hive/hplsql/Exec.java | 1950 ++++++++++++++++++
.../java/org/apache/hive/hplsql/Expression.java | 574 ++++++
.../main/java/org/apache/hive/hplsql/File.java | 132 ++
.../java/org/apache/hive/hplsql/Handler.java | 41 +
.../java/org/apache/hive/hplsql/Hplsql.java | 25 +
.../java/org/apache/hive/hplsql/Interval.java | 109 +
.../main/java/org/apache/hive/hplsql/Query.java | 155 ++
.../main/java/org/apache/hive/hplsql/Scope.java | 69 +
.../java/org/apache/hive/hplsql/Select.java | 411 ++++
.../java/org/apache/hive/hplsql/Signal.java | 48 +
.../main/java/org/apache/hive/hplsql/Stmt.java | 1021 +++++++++
.../org/apache/hive/hplsql/StreamGobbler.java | 51 +
.../main/java/org/apache/hive/hplsql/Timer.java | 59 +
.../main/java/org/apache/hive/hplsql/Udf.java | 117 ++
.../main/java/org/apache/hive/hplsql/Utils.java | 289 +++
.../main/java/org/apache/hive/hplsql/Var.java | 430 ++++
.../apache/hive/hplsql/functions/Function.java | 709 +++++++
.../hive/hplsql/functions/FunctionDatetime.java | 151 ++
.../hive/hplsql/functions/FunctionMisc.java | 188 ++
.../hive/hplsql/functions/FunctionOra.java | 231 +++
.../hive/hplsql/functions/FunctionString.java | 276 +++
pom.xml | 1 +
32 files changed, 9812 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/bin/ext/hplsql.sh
----------------------------------------------------------------------
diff --git a/bin/ext/hplsql.sh b/bin/ext/hplsql.sh
new file mode 100644
index 0000000..ddaf324
--- /dev/null
+++ b/bin/ext/hplsql.sh
@@ -0,0 +1,37 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+THISSERVICE=hplsql
+export SERVICE_LIST="${SERVICE_LIST}${THISSERVICE} "
+
+hplsql () {
+ CLASS=org.apache.hive.hplsql.Hplsql;
+
+ # include only the HPL/SQL jar and its dependencies
+ hplsqlJarPath=`ls ${HIVE_LIB}/hive-hplsql-*.jar`
+ antlrJarPath="${HIVE_LIB}/antlr-runtime-4.5.jar"
+ hadoopClasspath=""
+ if [[ -n "${HADOOP_CLASSPATH}" ]]
+ then
+ hadoopClasspath="${HADOOP_CLASSPATH}:"
+ fi
+ export HADOOP_CLASSPATH="${hadoopClasspath}${HIVE_CONF_DIR}:${hplsqlJarPath}:${antlrJarPath}"
+
+ exec $HADOOP jar ${hplsqlJarPath} $CLASS $HIVE_OPTS "$@"
+}
+
+hplsql_help () {
+ hplsql "--help"
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/bin/hplsql
----------------------------------------------------------------------
diff --git a/bin/hplsql b/bin/hplsql
new file mode 100644
index 0000000..6a5da7e
--- /dev/null
+++ b/bin/hplsql
@@ -0,0 +1,25 @@
+#!/usr/bin/env bash
+
+# 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.
+
+bin=`dirname "$0"`
+bin=`cd "$bin"; pwd`
+
+# Set Hadoop User classpath to true so that httpclient jars are taken from
+# hive lib instead of hadoop lib.
+export HADOOP_USER_CLASSPATH_FIRST=true
+
+. "$bin"/hive --service hplsql "$@"
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/bin/hplsql.cmd
----------------------------------------------------------------------
diff --git a/bin/hplsql.cmd b/bin/hplsql.cmd
new file mode 100644
index 0000000..6717a1c
--- /dev/null
+++ b/bin/hplsql.cmd
@@ -0,0 +1,58 @@
+@echo off
+@rem Licensed to the Apache Software Foundation (ASF) under one or more
+@rem contributor license agreements. See the NOTICE file distributed with
+@rem this work for additional information regarding copyright ownership.
+@rem The ASF licenses this file to You under the Apache License, Version 2.0
+@rem (the "License"); you may not use this file except in compliance with
+@rem the License. You may obtain a copy of the License at
+@rem
+@rem http://www.apache.org/licenses/LICENSE-2.0
+@rem
+@rem Unless required by applicable law or agreed to in writing, software
+@rem distributed under the License is distributed on an "AS IS" BASIS,
+@rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+@rem See the License for the specific language governing permissions and
+@rem limitations under the License.
+SetLocal EnableDelayedExpansion
+
+pushd %CD%\..
+if not defined HIVE_HOME (
+ set HIVE_HOME=%CD%
+)
+popd
+
+if "%HADOOP_BIN_PATH:~-1%" == "\" (
+ set HADOOP_BIN_PATH=%HADOOP_BIN_PATH:~0,-1%
+)
+
+if not defined JAVA_HOME (
+ echo Error: JAVA_HOME is not set.
+ goto :eof
+)
+
+@rem get the hadoop envrionment
+if not exist %HADOOP_HOME%\libexec\hadoop-config.cmd (
+ @echo +================================================================+
+ @echo ^| Error: HADOOP_HOME is not set correctly ^|
+ @echo +----------------------------------------------------------------+
+ @echo ^| Please set your HADOOP_HOME variable to the absolute path of ^|
+ @echo ^| the directory that contains \libexec\hadoop-config.cmd ^|
+ @echo +================================================================+
+ exit /b 1
+)
+@rem supress the HADOOP_HOME warnings in 1.x.x
+set HADOOP_HOME_WARN_SUPPRESS=true
+
+@rem include only the HPL/SQL jar and its dependencies
+pushd %HIVE_HOME%\lib
+for /f %%a IN ('dir /b hive-hplsql-**.jar') do (
+ set HADOOP_CLASSPATH=%HADOOP_CLASSPATH%;%HIVE_HOME%\lib\%%a
+)
+set HADOOP_CLASSPATH=%HADOOP_CLASSPATH%;%HIVE_HOME%\lib\antlr-runtime-4.5.jar
+popd
+set HADOOP_USER_CLASSPATH_FIRST=true
+call %HADOOP_HOME%\libexec\hadoop-config.cmd
+
+call "%JAVA_HOME%\bin\java" %JAVA_HEAP_MAX% %HADOOP_OPTS% -classpath %HADOOP_CLASSPATH% org.apache.hive.hplsql.Hplsql %*
+
+endlocal
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/pom.xml
----------------------------------------------------------------------
diff --git a/hplsql/pom.xml b/hplsql/pom.xml
new file mode 100644
index 0000000..d096e90
--- /dev/null
+++ b/hplsql/pom.xml
@@ -0,0 +1,123 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive</artifactId>
+ <version>2.0.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>hive-hplsql</artifactId>
+ <packaging>jar</packaging>
+ <name>Hive HPL/SQL</name>
+
+ <properties>
+ <hive.path.to.root>..</hive.path.to.root>
+ <packaging.minimizeJar>false</packaging.minimizeJar>
+ </properties>
+
+ <dependencies>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ <version>${guava.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-collections</groupId>
+ <artifactId>commons-collections</artifactId>
+ <version>3.2.1</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-cli</groupId>
+ <artifactId>commons-cli</artifactId>
+ <version>${commons-cli.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ <version>${commons-lang.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>commons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ <version>${commons-logging.version}</version>
+ </dependency>
+
+ <dependency>
+ <groupId>commons-io</groupId>
+ <artifactId>commons-io</artifactId>
+ <version>${commons-io.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.antlr</groupId>
+ <artifactId>antlr4-runtime</artifactId>
+ <version>4.5</version>
+ </dependency>
+ </dependencies>
+
+ <profiles>
+ <profile>
+ <id>hadoop-1</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-core</artifactId>
+ <version>${hadoop-20S.version}</version>
+ <optional>true</optional>
+ </dependency>
+ </dependencies>
+ </profile>
+ <profile>
+ <id>hadoop-2</id>
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <version>${hadoop-23.version}</version>
+ <optional>true</optional>
+ </dependency>
+ </dependencies>
+ </profile>
+ </profiles>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.antlr</groupId>
+ <artifactId>antlr4-maven-plugin</artifactId>
+ <version>4.5</version>
+ <configuration>
+ <visitor>true</visitor>
+ </configuration>
+ <executions>
+ <execution>
+ <goals>
+ <goal>antlr4</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
----------------------------------------------------------------------
diff --git a/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4 b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
new file mode 100644
index 0000000..852716b
--- /dev/null
+++ b/hplsql/src/main/antlr4/org/apache/hive/hplsql/Hplsql.g4
@@ -0,0 +1,1426 @@
+/**
+ 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.
+*/
+
+// HPL/SQL Procedural SQL Extension Grammar
+grammar Hplsql;
+
+program : block ;
+
+block : (begin_end_block | stmt)+ ; // Multiple consecutive blocks/statements
+
+begin_end_block :
+ declare_block? T_BEGIN block exception_block? T_END
+ ;
+
+single_block_stmt : // Single BEGIN END block (but nested blocks are possible) or single statement
+ T_BEGIN block T_END
+ | stmt T_SEMICOLON?
+ ;
+
+stmt :
+ assignment_stmt
+ | break_stmt
+ | call_stmt
+ | close_stmt
+ | copy_from_local_stmt
+ | copy_stmt
+ | commit_stmt
+ | create_function_stmt
+ | create_index_stmt
+ | create_local_temp_table_stmt
+ | create_procedure_stmt
+ | create_table_stmt
+ | declare_stmt
+ | delete_stmt
+ | drop_stmt
+ | exec_stmt
+ | exit_stmt
+ | fetch_stmt
+ | for_cursor_stmt
+ | for_range_stmt
+ | if_stmt
+ | include_stmt
+ | insert_stmt
+ | get_diag_stmt
+ | grant_stmt
+ | leave_stmt
+ | map_object_stmt
+ | merge_stmt
+ | open_stmt
+ | print_stmt
+ | resignal_stmt
+ | return_stmt
+ | rollback_stmt
+ | select_stmt
+ | signal_stmt
+ | update_stmt
+ | use_stmt
+ | values_into_stmt
+ | while_stmt
+ | label
+ | hive
+ | host
+ | expr_stmt
+ | semicolon_stmt // Placed here to allow null statements ;;...
+ ;
+
+semicolon_stmt :
+ T_SEMICOLON
+ | '@' | '#' | '/'
+ ;
+
+exception_block : // Exception block
+ T_EXCEPTION exception_block_item+
+ ;
+
+exception_block_item :
+ T_WHEN L_ID T_THEN block ~(T_WHEN | T_END)
+ ;
+
+expr_stmt : // Standalone expression
+ expr
+ ;
+
+assignment_stmt : // Assignment statement
+ T_SET? assignment_stmt_item (T_COMMA assignment_stmt_item)*
+ ;
+
+assignment_stmt_item :
+ assignment_stmt_single_item
+ | assignment_stmt_multiple_item
+ | assignment_stmt_select_item
+ ;
+
+assignment_stmt_single_item :
+ ident T_COLON? T_EQUAL expr
+ ;
+
+assignment_stmt_multiple_item :
+ T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P T_COLON? T_EQUAL T_OPEN_P expr (T_COMMA expr)* T_CLOSE_P
+ ;
+
+assignment_stmt_select_item :
+ (ident | (T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P)) T_COLON? T_EQUAL T_OPEN_P select_stmt T_CLOSE_P
+ ;
+
+break_stmt :
+ T_BREAK
+ ;
+
+call_stmt :
+ T_CALL ident expr_func_params?
+ ;
+
+declare_stmt : // Declaration statement
+ T_DECLARE declare_stmt_item (T_COMMA declare_stmt_item)*
+ ;
+
+declare_block : // Declaration block
+ T_DECLARE declare_stmt_item T_SEMICOLON (declare_stmt_item T_SEMICOLON)*
+ ;
+
+
+declare_stmt_item :
+ declare_var_item
+ | declare_condition_item
+ | declare_cursor_item
+ | declare_handler_item
+ | declare_temporary_table_item
+ ;
+
+declare_var_item :
+ ident (T_COMMA ident)* T_AS? dtype dtype_len? dtype_attr* dtype_default?
+ ;
+
+declare_condition_item : // Condition declaration
+ ident T_CONDITION
+ ;
+
+declare_cursor_item : // Cursor declaration
+ (T_CURSOR ident | ident T_CURSOR) declare_cursor_return? (T_IS | T_AS | T_FOR) (select_stmt | expr )
+ ;
+
+declare_cursor_return :
+ T_WITHOUT T_RETURN
+ | T_WITH T_RETURN T_ONLY? (T_TO (T_CALLER | T_CLIENT))?
+ ;
+
+declare_handler_item : // Condition handler declaration
+ (T_CONTINUE | T_EXIT) T_HANDLER T_FOR (T_SQLEXCEPTION | T_SQLWARNING | T_NOT T_FOUND | ident) single_block_stmt
+ ;
+
+declare_temporary_table_item : // DECLARE TEMPORARY TABLE statement
+ T_GLOBAL? T_TEMPORARY T_TABLE ident T_OPEN_P create_table_columns T_CLOSE_P create_table_options?
+ ;
+
+create_table_stmt :
+ T_CREATE T_TABLE (T_IF T_NOT T_EXISTS)? ident T_OPEN_P create_table_columns T_CLOSE_P create_table_options?
+ ;
+
+create_local_temp_table_stmt :
+ T_CREATE (T_LOCAL T_TEMPORARY | (T_SET | T_MULTISET)? T_VOLATILE) T_TABLE ident T_OPEN_P create_table_columns T_CLOSE_P create_table_options?
+ ;
+
+create_table_columns :
+ create_table_columns_item (T_COMMA create_table_columns_item)*
+ ;
+
+create_table_columns_item :
+ ident dtype dtype_len? dtype_attr* create_table_column_inline_cons?
+ | T_CONSTRAINT ident create_table_column_cons
+ ;
+
+create_table_column_inline_cons :
+ dtype_default
+ | T_NOT? T_NULL
+ | T_PRIMARY T_KEY
+ ;
+
+create_table_column_cons :
+ T_PRIMARY T_KEY T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P
+ ;
+
+create_table_options :
+ create_table_options_item+
+ ;
+
+create_table_options_item :
+ T_ON T_COMMIT (T_DELETE | T_PRESERVE) T_ROWS
+ | create_table_options_db2_item
+ | create_table_options_hive_item
+ ;
+
+create_table_options_db2_item :
+ T_IN ident
+ | T_WITH T_REPLACE
+ | T_DISTRIBUTE T_BY T_HASH T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P
+ | T_LOGGED
+ | T_NOT T_LOGGED
+ ;
+
+create_table_options_hive_item :
+ create_table_hive_row_format
+ ;
+
+create_table_hive_row_format :
+ T_ROW T_FORMAT T_DELIMITED create_table_hive_row_format_fields*
+ ;
+
+create_table_hive_row_format_fields :
+ T_FIELDS T_TERMINATED T_BY expr (T_ESCAPED T_BY expr)?
+ | T_COLLECTION T_ITEMS T_TERMINATED T_BY expr
+ | T_MAP T_KEYS T_TERMINATED T_BY expr
+ | T_LINES T_TERMINATED T_BY expr
+ | T_NULL T_DEFINED T_AS expr
+ ;
+
+dtype : // Data types
+ T_CHAR
+ | T_BIGINT
+ | T_DATE
+ | T_DEC
+ | T_DECIMAL
+ | T_FLOAT
+ | T_INT
+ | T_INTEGER
+ | T_NUMBER
+ | T_SMALLINT
+ | T_STRING
+ | T_TIMESTAMP
+ | T_VARCHAR
+ | T_VARCHAR2
+ | L_ID // User-defined data type
+ ;
+
+dtype_len : // Data type length or size specification
+ T_OPEN_P L_INT (T_COMMA L_INT)? T_CLOSE_P
+ ;
+
+dtype_attr :
+ T_CHARACTER T_SET ident
+ | T_NOT? (T_CASESPECIFIC | T_CS)
+ ;
+
+dtype_default : // Default clause in variable declaration
+ T_COLON? T_EQUAL expr
+ | T_DEFAULT expr
+ ;
+
+create_function_stmt :
+ (T_ALTER | T_CREATE (T_OR T_REPLACE)? | T_REPLACE) T_FUNCTION ident create_routine_params create_function_return (T_AS | T_IS)? single_block_stmt
+ ;
+
+create_function_return :
+ (T_RETURN | T_RETURNS) dtype dtype_len?
+ ;
+
+create_procedure_stmt :
+ (T_ALTER | T_CREATE (T_OR T_REPLACE)? | T_REPLACE) (T_PROCEDURE | T_PROC) ident create_routine_params create_routine_options? (T_AS | T_IS)? label? single_block_stmt (ident T_SEMICOLON)?
+ ;
+
+create_routine_params :
+ T_OPEN_P (create_routine_param_item (T_COMMA create_routine_param_item)*)? T_CLOSE_P
+ ;
+
+create_routine_param_item :
+ (T_IN | T_OUT | T_INOUT | T_IN T_OUT)? ident dtype dtype_len? dtype_attr* dtype_default?
+ | ident (T_IN | T_OUT | T_INOUT | T_IN T_OUT)? dtype dtype_len? dtype_attr* dtype_default?
+ ;
+
+create_routine_options :
+ create_routine_option+
+ ;
+create_routine_option :
+ T_LANGUAGE T_SQL
+ | T_SQL T_SECURITY (T_CREATOR | T_DEFINER | T_INVOKER | T_OWNER)
+ | T_DYNAMIC T_RESULT T_SETS L_INT
+ ;
+
+drop_stmt : // DROP statement
+ T_DROP T_TABLE (T_IF T_EXISTS)? table_name
+ ;
+
+exec_stmt : // EXEC, EXECUTE IMMEDIATE statement
+ (T_EXEC | T_EXECUTE) T_IMMEDIATE? expr (T_INTO L_ID (T_COMMA L_ID)*)? using_clause?
+ ;
+
+if_stmt : // IF statement
+ if_plsql_stmt
+ | if_tsql_stmt
+ ;
+
+if_plsql_stmt :
+ T_IF bool_expr T_THEN block elseif_block* else_block? T_END T_IF
+ ;
+
+if_tsql_stmt :
+ T_IF bool_expr single_block_stmt (T_ELSE single_block_stmt)?
+ ;
+
+elseif_block :
+ (T_ELSIF | T_ELSEIF) bool_expr T_THEN block
+ ;
+
+else_block :
+ T_ELSE block
+ ;
+
+include_stmt : // INCLUDE statement
+ T_INCLUDE file_name
+ ;
+
+insert_stmt : // INSERT statement
+ T_INSERT (T_OVERWRITE T_TABLE | T_INTO T_TABLE?) table_name insert_stmt_cols? (select_stmt | insert_stmt_rows)
+ ;
+
+insert_stmt_cols :
+ T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P
+ ;
+
+insert_stmt_rows :
+ T_VALUES insert_stmt_row (T_COMMA insert_stmt_row)*
+ ;
+
+insert_stmt_row:
+ T_OPEN_P expr (T_COMMA expr)* T_CLOSE_P
+ ;
+
+exit_stmt :
+ T_EXIT L_ID? (T_WHEN bool_expr)?
+ ;
+
+get_diag_stmt : // GET DIAGNOSTICS statement
+ T_GET T_DIAGNOSTICS get_diag_stmt_item
+ ;
+
+get_diag_stmt_item :
+ get_diag_stmt_exception_item
+ | get_diag_stmt_rowcount_item
+ ;
+
+get_diag_stmt_exception_item :
+ T_EXCEPTION L_INT ident T_EQUAL T_MESSAGE_TEXT
+ ;
+
+get_diag_stmt_rowcount_item :
+ ident T_EQUAL T_ROW_COUNT
+ ;
+
+grant_stmt :
+ T_GRANT grant_stmt_item (T_COMMA grant_stmt_item)* T_TO ident
+ ;
+
+grant_stmt_item :
+ T_EXECUTE T_ON T_PROCEDURE ident
+ ;
+
+leave_stmt :
+ T_LEAVE L_ID?
+ ;
+
+map_object_stmt :
+ T_MAP T_OBJECT expr (T_TO expr)? (T_AT expr)?
+ ;
+
+open_stmt : // OPEN cursor statement
+ T_OPEN L_ID (T_FOR (expr | select_stmt))?
+ ;
+
+fetch_stmt : // FETCH cursor statement
+ T_FETCH T_FROM? L_ID T_INTO L_ID (T_COMMA L_ID)*
+ ;
+
+close_stmt : // CLOSE cursor statement
+ T_CLOSE L_ID
+ ;
+
+copy_from_local_stmt : // COPY FROM LOCAL statement
+ T_COPY T_FROM T_LOCAL copy_source (T_COMMA copy_source)* T_TO copy_target copy_file_option*
+ ;
+
+copy_stmt : // COPY statement
+ T_COPY (table_name | T_OPEN_P select_stmt T_CLOSE_P) T_TO copy_target copy_option*
+ ;
+
+copy_source :
+ (ident | expr | L_FILE)
+ ;
+
+copy_target :
+ (ident | expr | L_FILE)
+ ;
+
+copy_option :
+ T_AT ident
+ | T_BATCHSIZE expr
+ | T_DELIMITER expr
+ | T_SQLINSERT ident
+ ;
+
+copy_file_option :
+ T_DELETE
+ | T_IGNORE
+ | T_OVERWRITE
+ ;
+
+commit_stmt : // COMMIT statement
+ T_COMMIT T_WORK?
+ ;
+
+create_index_stmt : // CREATE INDEX statement
+ T_CREATE T_UNIQUE? T_INDEX ident T_ON table_name T_OPEN_P create_index_col (T_COMMA create_index_col)* T_CLOSE_P
+ ;
+
+create_index_col :
+ ident (T_ASC | T_DESC)?
+ ;
+
+print_stmt : // PRINT statement
+ T_PRINT expr
+ | T_PRINT T_OPEN_P expr T_CLOSE_P
+ ;
+
+resignal_stmt : // RESIGNAL statement
+ T_RESIGNAL (T_SQLSTATE T_VALUE? expr (T_SET T_MESSAGE_TEXT T_EQUAL expr)? )?
+ ;
+
+return_stmt : // RETURN statement
+ T_RETURN expr?
+ ;
+
+rollback_stmt : // ROLLBACK statement
+ T_ROLLBACK T_WORK?
+ ;
+
+signal_stmt : // SIGNAL statement
+ T_SIGNAL ident
+ ;
+
+use_stmt : // USE statement
+ T_USE expr
+ ;
+
+values_into_stmt : // VALUES INTO statement
+ T_VALUES T_OPEN_P? expr (T_COMMA expr)* T_CLOSE_P? T_INTO T_OPEN_P? ident (T_COMMA ident)* T_CLOSE_P?
+ ;
+
+while_stmt : // WHILE loop statement
+ T_WHILE bool_expr (T_DO | T_LOOP | T_THEN | T_BEGIN) block T_END (T_WHILE | T_LOOP)?
+ ;
+
+for_cursor_stmt : // FOR (cursor) statement
+ T_FOR L_ID T_IN T_OPEN_P? select_stmt T_CLOSE_P? T_LOOP block T_END T_LOOP
+ ;
+
+for_range_stmt : // FOR (Integer range) statement
+ T_FOR L_ID T_IN T_REVERSE? expr T_DOT2 expr ((T_BY | T_STEP) expr)? T_LOOP block T_END T_LOOP
+ ;
+
+label :
+ L_LABEL
+ | T_LESS T_LESS L_ID T_GREATER T_GREATER
+ ;
+
+using_clause : // USING var,... clause
+ T_USING expr (T_COMMA expr)*
+ ;
+
+select_stmt : // SELECT statement
+ cte_select_stmt? fullselect_stmt
+ ;
+
+cte_select_stmt :
+ T_WITH cte_select_stmt_item (T_COMMA cte_select_stmt_item)*
+ ;
+
+cte_select_stmt_item :
+ ident cte_select_cols? T_AS T_OPEN_P fullselect_stmt T_CLOSE_P
+ ;
+
+cte_select_cols :
+ T_OPEN_P ident (T_COMMA ident)* T_CLOSE_P
+ ;
+
+fullselect_stmt :
+ fullselect_stmt_item (fullselect_set_clause fullselect_stmt_item)*
+ ;
+
+fullselect_stmt_item :
+ subselect_stmt
+ | T_OPEN_P fullselect_stmt T_CLOSE_P
+ ;
+
+fullselect_set_clause :
+ T_UNION T_ALL?
+ | T_EXCEPT T_ALL?
+ | T_INTERSECT T_ALL?
+ ;
+
+subselect_stmt :
+ (T_SELECT | T_SEL) select_list into_clause? from_clause? where_clause? group_by_clause? having_clause? order_by_clause? select_options?
+ ;
+
+select_list :
+ select_list_set? select_list_limit? select_list_item (T_COMMA select_list_item)*
+ ;
+
+select_list_set :
+ T_ALL
+ | T_DISTINCT
+ ;
+
+select_list_limit :
+ T_TOP expr
+ ;
+
+select_list_item :
+ (expr select_list_alias? | select_list_asterisk)
+ ;
+
+select_list_alias :
+ T_AS? L_ID
+ | T_OPEN_P T_TITLE L_S_STRING T_CLOSE_P
+ ;
+
+select_list_asterisk :
+ (L_ID '.')? '*'
+ ;
+
+into_clause :
+ T_INTO ident (T_COMMA ident)*
+ ;
+
+from_clause :
+ T_FROM from_table_clause (from_join_clause)*
+ ;
+
+from_table_clause :
+ from_table_name_clause
+ | from_subselect_clause
+ | from_table_values_clause
+ ;
+
+from_table_name_clause :
+ table_name from_alias_clause?
+ ;
+
+from_subselect_clause :
+ T_OPEN_P subselect_stmt T_CLOSE_P from_alias_clause?
+ ;
+
+from_join_clause :
+ T_COMMA from_table_clause
+ | from_join_type_clause from_table_clause T_ON bool_expr
+ ;
+
+from_join_type_clause :
+ T_INNER T_JOIN
+ | (T_LEFT | T_RIGHT | T_FULL) T_OUTER? T_JOIN
+ ;
+
+from_table_values_clause:
+ T_TABLE T_OPEN_P T_VALUES from_table_values_row (T_COMMA from_table_values_row)* T_CLOSE_P from_alias_clause?
+ ;
+
+from_table_values_row:
+ expr
+ | T_OPEN_P expr (T_COMMA expr)* T_CLOSE_P
+ ;
+
+from_alias_clause :
+ {!_input.LT(1).getText().equalsIgnoreCase("GROUP") &&
+ !_input.LT(1).getText().equalsIgnoreCase("ORDER") &&
+ !_input.LT(1).getText().equalsIgnoreCase("LIMIT")}?
+ T_AS? ident (T_OPEN_P L_ID (T_COMMA L_ID)* T_CLOSE_P)?
+ ;
+
+table_name :
+ ident
+ ;
+
+where_clause :
+ T_WHERE bool_expr
+ ;
+
+group_by_clause :
+ T_GROUP T_BY expr (T_COMMA expr)*
+ ;
+
+having_clause :
+ T_HAVING bool_expr
+ ;
+
+order_by_clause :
+ T_ORDER T_BY expr (T_ASC | T_DESC)? (T_COMMA expr (T_ASC | T_DESC)?)*
+ ;
+
+select_options :
+ select_options_item+
+ ;
+
+select_options_item :
+ T_LIMIT expr
+ | T_WITH (T_RR | T_RS | T_CS | T_UR)
+ ;
+
+update_stmt : // UPDATE statement
+ T_UPDATE update_table T_SET assignment_stmt_item (T_COMMA assignment_stmt_item)* where_clause? update_upsert?
+ ;
+
+update_table :
+ (table_name | (T_OPEN_P select_stmt T_CLOSE_P)) (T_AS? ident)?
+ ;
+
+update_upsert :
+ T_ELSE insert_stmt
+ ;
+
+merge_stmt : // MERGE statement
+ T_MERGE T_INTO merge_table T_USING merge_table T_ON bool_expr merge_condition+
+ ;
+
+merge_table :
+ (table_name | (T_OPEN_P select_stmt T_CLOSE_P)) (T_AS? ident)?
+ ;
+
+merge_condition :
+ T_WHEN T_NOT? T_MATCHED (T_AND bool_expr)? T_THEN merge_action
+ | T_ELSE T_IGNORE
+ ;
+
+merge_action :
+ T_INSERT insert_stmt_cols? T_VALUES insert_stmt_row
+ | T_UPDATE T_SET assignment_stmt_item (T_COMMA assignment_stmt_item)*
+ | T_DELETE
+ ;
+
+delete_stmt : // DELETE statement
+ T_DELETE T_FROM? table_name (T_AS? ident)? where_clause?
+ ;
+
+bool_expr : // Boolean condition
+ T_OPEN_P bool_expr T_CLOSE_P
+ | bool_expr bool_expr_logical_operator bool_expr
+ | bool_expr_atom
+ ;
+
+bool_expr_atom :
+ bool_expr_unary
+ | bool_expr_binary
+ ;
+
+bool_expr_unary :
+ expr T_IS T_NOT? T_NULL
+ | expr T_BETWEEN expr T_AND expr
+ | bool_expr_single_in
+ | bool_expr_multi_in
+ ;
+
+bool_expr_single_in :
+ expr T_NOT? T_IN T_OPEN_P ((expr (T_COMMA expr)*) | select_stmt) T_CLOSE_P
+ ;
+
+bool_expr_multi_in :
+ T_OPEN_P expr (T_COMMA expr)* T_CLOSE_P T_NOT? T_IN T_OPEN_P select_stmt T_CLOSE_P
+ ;
+
+bool_expr_binary :
+ expr bool_expr_binary_operator expr
+ ;
+
+bool_expr_logical_operator :
+ T_AND
+ | T_OR
+ ;
+
+bool_expr_binary_operator :
+ T_EQUAL
+ | T_EQUAL2
+ | T_NOTEQUAL
+ | T_NOTEQUAL2
+ | T_LESS
+ | T_LESSEQUAL
+ | T_GREATER
+ | T_GREATEREQUAL
+ | T_NOT? (T_LIKE | T_RLIKE | T_REGEXP)
+ ;
+
+expr :
+ expr interval_item
+ | expr T_MUL expr
+ | expr T_DIV expr
+ | expr T_ADD expr
+ | expr T_SUB expr
+ | T_OPEN_P expr T_CLOSE_P
+ | expr_concat
+ | expr_case
+ | expr_agg_window_func
+ | expr_spec_func
+ | expr_func
+ | expr_atom
+ ;
+
+
+expr_atom :
+ date_literal
+ | timestamp_literal
+ | ident
+ | string
+ | dec_number
+ | interval_number
+ | int_number
+ | null_const
+ ;
+
+interval_item :
+ T_DAY
+ | T_DAYS
+ | T_MICROSECOND
+ | T_MICROSECONDS
+ ;
+
+interval_number :
+ int_number interval_item
+ ;
+
+expr_concat : // String concatenation operator
+ expr_concat_item (T_PIPE | T_CONCAT) expr_concat_item ((T_PIPE | T_CONCAT) expr_concat_item)*
+ ;
+
+expr_concat_item :
+ T_OPEN_P expr T_CLOSE_P
+ | expr_case
+ | expr_agg_window_func
+ | expr_spec_func
+ | expr_func
+ | expr_atom
+ ;
+
+expr_case : // CASE expression
+ expr_case_simple
+ | expr_case_searched
+ ;
+
+expr_case_simple :
+ T_CASE expr (T_WHEN expr T_THEN expr)+ (T_ELSE expr)? T_END
+ ;
+
+expr_case_searched :
+ T_CASE (T_WHEN bool_expr T_THEN expr)+ (T_ELSE expr)? T_END
+ ;
+
+expr_agg_window_func :
+ T_AVG T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
+ | T_COUNT T_OPEN_P ((expr_func_all_distinct? expr) | '*') T_CLOSE_P expr_func_over_clause?
+ | T_COUNT_BIG T_OPEN_P ((expr_func_all_distinct? expr) | '*') T_CLOSE_P expr_func_over_clause?
+ | T_DENSE_RANK T_OPEN_P T_CLOSE_P expr_func_over_clause
+ | T_FIRST_VALUE T_OPEN_P expr T_CLOSE_P expr_func_over_clause
+ | T_LAG T_OPEN_P expr (T_COMMA expr (T_COMMA expr)?)? T_CLOSE_P expr_func_over_clause
+ | T_LAST_VALUE T_OPEN_P expr T_CLOSE_P expr_func_over_clause
+ | T_LEAD T_OPEN_P expr (T_COMMA expr (T_COMMA expr)?)? T_CLOSE_P expr_func_over_clause
+ | T_MAX T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
+ | T_MIN T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
+ | T_RANK T_OPEN_P T_CLOSE_P expr_func_over_clause
+ | T_ROW_NUMBER T_OPEN_P T_CLOSE_P expr_func_over_clause
+ | T_STDEV T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
+ | T_SUM T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
+ | T_VAR T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
+ | T_VARIANCE T_OPEN_P expr_func_all_distinct? expr T_CLOSE_P expr_func_over_clause?
+ ;
+
+expr_func_all_distinct :
+ T_ALL
+ | T_DISTINCT
+ ;
+
+expr_func_over_clause :
+ T_OVER T_OPEN_P expr_func_partition_by_clause? order_by_clause? T_CLOSE_P
+ ;
+
+expr_func_partition_by_clause :
+ T_PARTITION T_BY ident (T_COMMA ident)*
+ ;
+
+expr_spec_func :
+ T_ACTIVITY_COUNT
+ | T_CAST T_OPEN_P expr T_AS dtype dtype_len? T_CLOSE_P
+ | T_COUNT T_OPEN_P (expr | '*') T_CLOSE_P
+ | T_CURRENT_DATE | T_CURRENT T_DATE
+ | (T_CURRENT_TIMESTAMP | T_CURRENT T_TIMESTAMP) (T_OPEN_P expr T_CLOSE_P)?
+ | T_CURRENT_USER | T_CURRENT T_USER
+ | T_MAX_PART_STRING T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P
+ | T_MIN_PART_STRING T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P
+ | T_MAX_PART_INT T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P
+ | T_MIN_PART_INT T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P
+ | T_MAX_PART_DATE T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P
+ | T_MIN_PART_DATE T_OPEN_P expr (T_COMMA expr (T_COMMA expr T_EQUAL expr)*)? T_CLOSE_P
+ | T_PART_LOC T_OPEN_P expr (T_COMMA expr T_EQUAL expr)+ (T_COMMA expr)? T_CLOSE_P
+ | T_TRIM T_OPEN_P expr T_CLOSE_P
+ | T_SUBSTRING T_OPEN_P expr T_FROM expr (T_FOR expr)? T_CLOSE_P
+ | T_SYSDATE
+ | T_USER
+ ;
+
+expr_func :
+ ident expr_func_params
+ ;
+
+expr_func_params :
+ T_OPEN_P (expr (T_COMMA expr)*)? T_CLOSE_P
+ ;
+
+hive :
+ T_HIVE hive_item*
+ ;
+
+hive_item :
+ P_e expr
+ | P_f expr
+ | P_hiveconf L_ID T_EQUAL expr
+ | P_i expr
+ | P_S
+ | P_h
+ ;
+
+host :
+ '!' host_cmd ';' // OS command
+ | host_stmt
+ ;
+
+host_cmd :
+ .*?
+ ;
+
+host_stmt :
+ T_HOST expr
+ ;
+
+file_name :
+ L_ID | L_FILE
+ ;
+
+date_literal : // DATE 'YYYY-MM-DD' literal
+ T_DATE string
+ ;
+
+timestamp_literal : // TIMESTAMP 'YYYY-MM-DD HH:MI:SS.FFF' literal
+ T_TIMESTAMP string
+ ;
+
+ident :
+ L_ID
+ | non_reserved_words
+ ;
+
+string : // String literal (single or double quoted)
+ L_S_STRING # single_quotedString
+ | L_D_STRING # double_quotedString
+ ;
+
+int_number : // Integer (positive or negative)
+ ('-' | '+')? L_INT
+ ;
+
+dec_number : // Decimal number (positive or negative)
+ ('-' | '+')? L_DEC
+ ;
+
+null_const : // NULL constant
+ T_NULL
+ ;
+
+non_reserved_words : // Tokens that are not reserved words and can be used as identifiers
+ T_ACTIVITY_COUNT
+ | T_ALL
+ | T_ALTER
+ | T_AND
+ | T_AS
+ | T_ASC
+ | T_AT
+ | T_AVG
+ | T_BATCHSIZE
+ | T_BEGIN
+ | T_BETWEEN
+ | T_BIGINT
+ | T_BREAK
+ | T_BY
+ | T_CALL
+ | T_CALLER
+ | T_CASE
+ | T_CASESPECIFIC
+ | T_CAST
+ | T_CHAR
+ | T_CHARACTER
+ | T_CLIENT
+ | T_CLOSE
+ | T_COLLECTION
+ | T_COPY
+ | T_COMMIT
+ | T_CONCAT
+ | T_CONDITION
+ | T_CONSTRAINT
+ | T_CONTINUE
+ | T_COUNT
+ | T_COUNT_BIG
+ | T_CREATE
+ | T_CREATOR
+ | T_CS
+ | T_CURRENT
+ | T_CURRENT_DATE
+ | T_CURRENT_TIMESTAMP
+ | T_CURRENT_USER
+ | T_CURSOR
+ | T_DATE
+ | T_DAY
+ | T_DAYS
+ | T_DEC
+ | T_DECIMAL
+ | T_DECLARE
+ | T_DEFAULT
+ | T_DEFINED
+ | T_DEFINER
+ | T_DELETE
+ | T_DELIMITED
+ | T_DELIMITER
+ | T_DENSE_RANK
+ | T_DESC
+ | T_DIAGNOSTICS
+ | T_DISTINCT
+ | T_DISTRIBUTE
+ | T_DO
+ | T_DROP
+ | T_DYNAMIC
+ // T_ELSE reserved word
+ // T_ELSEIF reserved word
+ // T_ELSIF reserved word
+ // T_END reserved word
+ | T_ESCAPED
+ | T_EXCEPT
+ | T_EXEC
+ | T_EXECUTE
+ | T_EXCEPTION
+ | T_EXISTS
+ | T_EXIT
+ | T_FETCH
+ | T_FIELDS
+ | T_FILE
+ | T_FIRST_VALUE
+ | T_FLOAT
+ | T_FOR
+ | T_FORMAT
+ | T_FOUND
+ | T_FROM
+ | T_FULL
+ | T_FUNCTION
+ | T_GET
+ | T_GLOBAL
+ | T_GRANT
+ | T_GROUP
+ | T_HANDLER
+ | T_HASH
+ | T_HAVING
+ | T_HIVE
+ | T_HOST
+ | T_IF
+ | T_IGNORE
+ | T_IMMEDIATE
+ | T_IN
+ | T_INCLUDE
+ | T_INDEX
+ | T_INNER
+ | T_INOUT
+ | T_INSERT
+ | T_INT
+ | T_INTEGER
+ | T_INTERSECT
+ | T_INTO
+ | T_INVOKER
+ | T_ITEMS
+ | T_IS
+ | T_JOIN
+ | T_KEY
+ | T_KEYS
+ | T_LAG
+ | T_LANGUAGE
+ | T_LAST_VALUE
+ | T_LEAD
+ | T_LEAVE
+ | T_LEFT
+ | T_LIKE
+ | T_LIMIT
+ | T_LINES
+ | T_LOCAL
+ | T_LOGGED
+ | T_LOOP
+ | T_MAP
+ | T_MATCHED
+ | T_MAX
+ | T_MERGE
+ | T_MESSAGE_TEXT
+ | T_MICROSECOND
+ | T_MICROSECONDS
+ | T_MIN
+ | T_MULTISET
+ | T_NOT
+ // T_NULL reserved word
+ | T_NUMBER
+ | T_OBJECT
+ | T_ON
+ | T_ONLY
+ | T_OPEN
+ | T_OR
+ | T_ORDER
+ | T_OUT
+ | T_OUTER
+ | T_OVER
+ | T_OVERWRITE
+ | T_OWNER
+ | T_PART_LOC
+ | T_PARTITION
+ | T_PRESERVE
+ | T_PRIMARY
+ | T_PRINT
+ | T_PROC
+ | T_PROCEDURE
+ | T_RANK
+ | T_REGEXP
+ | T_RR
+ | T_REPLACE
+ | T_RESIGNAL
+ | T_RESULT
+ | T_RETURN
+ | T_RETURNS
+ | T_REVERSE
+ | T_RIGHT
+ | T_RLIKE
+ | T_RS
+ | T_ROLLBACK
+ | T_ROW
+ | T_ROWS
+ | T_ROW_COUNT
+ | T_ROW_NUMBER
+ | T_SECURITY
+ | T_SEL
+ | T_SELECT
+ | T_SET
+ | T_SETS
+ | T_SIGNAL
+ | T_SMALLINT
+ | T_SQL
+ | T_SQLEXCEPTION
+ | T_SQLINSERT
+ | T_SQLSTATE
+ | T_SQLWARNING
+ | T_STEP
+ | T_STDEV
+ | T_STRING
+ | T_SUBSTRING
+ | T_SUM
+ | T_SYSDATE
+ | T_TABLE
+ | T_TEMPORARY
+ | T_TERMINATED
+ | T_THEN
+ | T_TIMESTAMP
+ | T_TITLE
+ | T_TO
+ | T_TOP
+ | T_TRIM
+ // T_UNION reserved word
+ | T_UNIQUE
+ | T_UPDATE
+ | T_UR
+ | T_USE
+ | T_USER
+ | T_USING
+ | T_VALUE
+ | T_VALUES
+ | T_VAR
+ | T_VARCHAR
+ | T_VARCHAR2
+ | T_VARIANCE
+ | T_VOLATILE
+ // T_WHEN reserved word
+ // T_WHERE reserved word
+ | T_WHILE
+ | T_WITH
+ | T_WITHOUT
+ | T_WORK
+ ;
+
+// Lexer rules
+T_ALL : A L L ;
+T_ALTER : A L T E R ;
+T_AND : A N D ;
+T_AS : A S ;
+T_ASC : A S C ;
+T_AT : A T ;
+T_AVG : A V G ;
+T_BATCHSIZE : B A T C H S I Z E ;
+T_BEGIN : B E G I N ;
+T_BETWEEN : B E T W E E N ;
+T_BIGINT : B I G I N T ;
+T_BREAK : B R E A K ;
+T_BY : B Y ;
+T_CALL : C A L L ;
+T_CALLER : C A L L E R ;
+T_CASE : C A S E ;
+T_CASESPECIFIC : C A S E S P E C I F I C ;
+T_CAST : C A S T ;
+T_CHAR : C H A R ;
+T_CHARACTER : C H A R A C T E R ;
+T_CLIENT : C L I E N T ;
+T_CLOSE : C L O S E ;
+T_COLLECTION : C O L L E C T I O N ;
+T_COPY : C O P Y ;
+T_COMMIT : C O M M I T ;
+T_CONCAT : C O N C A T;
+T_CONDITION : C O N D I T I O N ;
+T_CONSTRAINT : C O N S T R A I N T ;
+T_CONTINUE : C O N T I N U E ;
+T_COUNT : C O U N T ;
+T_COUNT_BIG : C O U N T '_' B I G;
+T_CREATE : C R E A T E ;
+T_CREATOR : C R E A T O R ;
+T_CS : C S;
+T_CURRENT : C U R R E N T ;
+T_CURSOR : C U R S O R ;
+T_DATE : D A T E ;
+T_DAY : D A Y ;
+T_DAYS : D A Y S ;
+T_DEC : D E C ;
+T_DECIMAL : D E C I M A L ;
+T_DECLARE : D E C L A R E ;
+T_DEFAULT : D E F A U L T ;
+T_DEFINED : D E F I N E D ;
+T_DEFINER : D E F I N E R ;
+T_DELETE : D E L E T E ;
+T_DELIMITED : D E L I M I T E D ;
+T_DELIMITER : D E L I M I T E R ;
+T_DESC : D E S C ;
+T_DIAGNOSTICS : D I A G N O S T I C S ;
+T_DISTINCT : D I S T I N C T ;
+T_DISTRIBUTE : D I S T R I B U T E ;
+T_DO : D O ;
+T_DROP : D R O P ;
+T_DYNAMIC : D Y N A M I C ;
+T_ELSE : E L S E ;
+T_ELSEIF : E L S E I F ;
+T_ELSIF : E L S I F ;
+T_END : E N D ;
+T_ESCAPED : E S C A P E D ;
+T_EXCEPT : E X C E P T ;
+T_EXEC : E X E C ;
+T_EXECUTE : E X E C U T E ;
+T_EXCEPTION : E X C E P T I O N ;
+T_EXISTS : E X I S T S ;
+T_EXIT : E X I T ;
+T_FETCH : F E T C H ;
+T_FIELDS : F I E L D S ;
+T_FILE : F I L E ;
+T_FLOAT : F L O A T ;
+T_FOR : F O R ;
+T_FORMAT : F O R M A T ;
+T_FOUND : F O U N D ;
+T_FROM : F R O M ;
+T_FULL : F U L L ;
+T_FUNCTION : F U N C T I O N ;
+T_GET : G E T ;
+T_GLOBAL : G L O B A L ;
+T_GRANT : G R A N T ;
+T_GROUP : G R O U P ;
+T_HANDLER : H A N D L E R ;
+T_HASH : H A S H ;
+T_HAVING : H A V I N G ;
+T_HIVE : H I V E ;
+T_HOST : H O S T ;
+T_IF : I F ;
+T_IGNORE : I G N O R E ;
+T_IMMEDIATE : I M M E D I A T E ;
+T_IN : I N ;
+T_INCLUDE : I N C L U D E ;
+T_INDEX : I N D E X ;
+T_INNER : I N N E R ;
+T_INOUT : I N O U T;
+T_INSERT : I N S E R T ;
+T_INT : I N T ;
+T_INTEGER : I N T E G E R ;
+T_INTERSECT : I N T E R S E C T ;
+T_INTO : I N T O ;
+T_INVOKER : I N V O K E R ;
+T_IS : I S ;
+T_ITEMS : I T E M S ;
+T_JOIN : J O I N ;
+T_KEY : K E Y ;
+T_KEYS : K E Y S ;
+T_LANGUAGE : L A N G U A G E ;
+T_LEAVE : L E A V E ;
+T_LEFT : L E F T ;
+T_LIKE : L I K E ;
+T_LIMIT : L I M I T ;
+T_LINES : L I N E S ;
+T_LOCAL : L O C A L ;
+T_LOGGED : L O G G E D ;
+T_LOOP : L O O P ;
+T_MAP : M A P ;
+T_MATCHED : M A T C H E D ;
+T_MAX : M A X ;
+T_MERGE : M E R G E ;
+T_MESSAGE_TEXT : M E S S A G E '_' T E X T ;
+T_MICROSECOND : M I C R O S E C O N D ;
+T_MICROSECONDS : M I C R O S E C O N D S;
+T_MIN : M I N ;
+T_MULTISET : M U L T I S E T ;
+T_NOT : N O T ;
+T_NULL : N U L L ;
+T_NUMBER : N U M B E R ;
+T_OBJECT : O B J E C T ;
+T_ON : O N ;
+T_ONLY : O N L Y ;
+T_OPEN : O P E N ;
+T_OR : O R ;
+T_ORDER : O R D E R;
+T_OUT : O U T ;
+T_OUTER : O U T E R ;
+T_OVER : O V E R ;
+T_OVERWRITE : O V E R W R I T E ;
+T_OWNER : O W N E R ;
+T_PARTITION : P A R T I T I O N ;
+T_PRESERVE : P R E S E R V E ;
+T_PRIMARY : P R I M A R Y ;
+T_PRINT : P R I N T ;
+T_PROC : P R O C ;
+T_PROCEDURE : P R O C E D U R E;
+T_REGEXP : R E G E X P ;
+T_REPLACE : R E P L A C E ;
+T_RESIGNAL : R E S I G N A L ;
+T_RESULT : R E S U L T ;
+T_RETURN : R E T U R N ;
+T_RETURNS : R E T U R N S ;
+T_REVERSE : R E V E R S E ;
+T_RIGHT : R I G H T ;
+T_RLIKE : R L I K E ;
+T_ROLLBACK : R O L L B A C K ;
+T_ROW : R O W ;
+T_ROWS : R O W S ;
+T_ROW_COUNT : R O W '_' C O U N T ;
+T_RR : R R;
+T_RS : R S ;
+T_TRIM : T R I M ;
+T_SECURITY : S E C U R I T Y ;
+T_SEL : S E L ;
+T_SELECT : S E L E C T ;
+T_SET : S E T ;
+T_SETS : S E T S;
+T_SIGNAL : S I G N A L ;
+T_SMALLINT : S M A L L I N T ;
+T_SQL : S Q L ;
+T_SQLEXCEPTION : S Q L E X C E P T I O N ;
+T_SQLINSERT : S Q L I N S E R T ;
+T_SQLSTATE : S Q L S T A T E ;
+T_SQLWARNING : S Q L W A R N I N G ;
+T_STEP : S T E P ;
+T_STRING : S T R I N G ;
+T_SUBSTRING : S U B S T R I N G ;
+T_SUM : S U M ;
+T_TABLE : T A B L E ;
+T_TEMPORARY : T E M P O R A R Y ;
+T_TERMINATED : T E R M I N A T E D ;
+T_THEN : T H E N ;
+T_TIMESTAMP : T I M E S T A M P ;
+T_TITLE : T I T L E ;
+T_TO : T O ;
+T_TOP : T O P ;
+T_UNION : U N I O N ;
+T_UNIQUE : U N I Q U E ;
+T_UPDATE : U P D A T E ;
+T_UR : U R ;
+T_USE : U S E ;
+T_USING : U S I N G ;
+T_VALUE : V A L U E ;
+T_VALUES : V A L U E S ;
+T_VAR : V A R ;
+T_VARCHAR : V A R C H A R ;
+T_VARCHAR2 : V A R C H A R '2' ;
+T_VOLATILE : V O L A T I L E ;
+T_WHEN : W H E N ;
+T_WHERE : W H E R E ;
+T_WHILE : W H I L E ;
+T_WITH : W I T H ;
+T_WITHOUT : W I T H O U T ;
+T_WORK : W O R K ;
+
+// Functions with specific syntax
+T_ACTIVITY_COUNT : A C T I V I T Y '_' C O U N T ;
+T_CURRENT_DATE : C U R R E N T '_' D A T E ;
+T_CURRENT_TIMESTAMP : C U R R E N T '_' T I M E S T A M P ;
+T_CURRENT_USER : C U R R E N T '_' U S E R ;
+T_DENSE_RANK : D E N S E '_' R A N K ;
+T_FIRST_VALUE : F I R S T '_' V A L U E;
+T_LAG : L A G ;
+T_LAST_VALUE : L A S T '_' V A L U E;
+T_LEAD : L E A D ;
+T_MAX_PART_STRING : M A X '_' P A R T '_' S T R I N G ;
+T_MIN_PART_STRING : M I N '_' P A R T '_' S T R I N G ;
+T_MAX_PART_INT : M A X '_' P A R T '_' I N T ;
+T_MIN_PART_INT : M I N '_' P A R T '_' I N T ;
+T_MAX_PART_DATE : M A X '_' P A R T '_' D A T E ;
+T_MIN_PART_DATE : M I N '_' P A R T '_' D A T E ;
+T_PART_LOC : P A R T '_' L O C ;
+T_RANK : R A N K ;
+T_ROW_NUMBER : R O W '_' N U M B E R;
+T_STDEV : S T D E V ;
+T_SYSDATE : S Y S D A T E ;
+T_VARIANCE : V A R I A N C E ;
+T_USER : U S E R;
+
+T_ADD : '+' ;
+T_COLON : ':' ;
+T_COMMA : ',' ;
+T_PIPE : '||' ;
+T_DIV : '/' ;
+T_DOT2 : '..' ;
+T_EQUAL : '=' ;
+T_EQUAL2 : '==' ;
+T_NOTEQUAL : '<>' ;
+T_NOTEQUAL2 : '!=' ;
+T_GREATER : '>' ;
+T_GREATEREQUAL : '>=' ;
+T_LESS : '<' ;
+T_LESSEQUAL : '<=' ;
+T_MUL : '*' ;
+T_OPEN_B : '{' ;
+T_OPEN_P : '(' ;
+T_CLOSE_B : '}' ;
+T_CLOSE_P : ')' ;
+T_SEMICOLON : ';' ;
+T_SUB : '-' ;
+
+P_e : '-e' ;
+P_f : '-f' ;
+P_hiveconf : '-hiveconf' ;
+P_i : '-i' ;
+P_S : '-S' ;
+P_h : '-h' ;
+
+L_ID : L_ID_PART (L_BLANK* '.' L_BLANK* L_ID_PART)* // Identifier
+ ;
+L_S_STRING : '\'' (('\'' '\'') | ('\\' '\'') | ~('\''))* '\'' // Single quoted string literal
+ ;
+L_D_STRING : '"' (L_STR_ESC_D | .)*? '"' // Double quoted string literal
+ ;
+L_INT : L_DIGIT+ ; // Integer
+L_DEC : L_DIGIT+ '.' ~'.' L_DIGIT* // Decimal number
+ | '.' L_DIGIT+
+ ;
+L_WS : L_BLANK+ -> skip ; // Whitespace
+L_M_COMMENT : '/*' .*? '*/' -> channel(HIDDEN) ; // Multiline comment
+L_S_COMMENT : ('--' | '//') .*? '\r'? '\n' -> channel(HIDDEN) ; // Single line comment
+
+L_FILE : '/'? L_ID ('/' L_ID)* // File path
+ | ([a-zA-Z] ':' '\\'?)? L_ID ('\\' L_ID)*
+ ;
+
+L_LABEL : ([a-zA-Z] | L_DIGIT | '_')* ':'
+ ;
+
+fragment
+L_ID_PART :
+ [a-zA-Z] ([a-zA-Z] | L_DIGIT | '_')* // Identifier part
+ | ('_' | '@' | ':' | '#' | '$') ([a-zA-Z] | L_DIGIT | '_' | '@' | ':' | '#' | '$')+ // (at least one char must follow special char)
+ | '"' .*? '"' // Quoted identifiers
+ | '[' .*? ']'
+ | '`' .*? '`'
+ ;
+fragment
+L_STR_ESC_D : // Double quoted string escape sequence
+ '""' | '\\"'
+ ;
+fragment
+L_DIGIT : [0-9] // Digit
+ ;
+fragment
+L_BLANK : (' ' | '\t' | '\r' | '\n')
+ ;
+
+// Support case-insensitive keywords and allowing case-sensitive identifiers
+fragment A : ('a'|'A') ;
+fragment B : ('b'|'B') ;
+fragment C : ('c'|'C') ;
+fragment D : ('d'|'D') ;
+fragment E : ('e'|'E') ;
+fragment F : ('f'|'F') ;
+fragment G : ('g'|'G') ;
+fragment H : ('h'|'H') ;
+fragment I : ('i'|'I') ;
+fragment J : ('j'|'J') ;
+fragment K : ('k'|'K') ;
+fragment L : ('l'|'L') ;
+fragment M : ('m'|'M') ;
+fragment N : ('n'|'N') ;
+fragment O : ('o'|'O') ;
+fragment P : ('p'|'P') ;
+fragment Q : ('q'|'Q') ;
+fragment R : ('r'|'R') ;
+fragment S : ('s'|'S') ;
+fragment T : ('t'|'T') ;
+fragment U : ('u'|'U') ;
+fragment V : ('v'|'V') ;
+fragment W : ('w'|'W') ;
+fragment X : ('x'|'X') ;
+fragment Y : ('y'|'Y') ;
+fragment Z : ('z'|'Z') ;
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Arguments.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Arguments.java b/hplsql/src/main/java/org/apache/hive/hplsql/Arguments.java
new file mode 100644
index 0000000..604d9a7
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Arguments.java
@@ -0,0 +1,206 @@
+/**
+ * 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.hive.hplsql;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.ParseException;
+
+public class Arguments {
+ private CommandLine commandLine;
+ private Options options = new Options();
+
+ String execString;
+ String fileName;
+ String main;
+ Map<String, String> vars = new HashMap<String, String>();
+
+ @SuppressWarnings("static-access")
+ Arguments() {
+ // -e 'query'
+ options.addOption(OptionBuilder
+ .hasArg()
+ .withArgName("quoted-query-string")
+ .withDescription("HPL/SQL from command line")
+ .create('e'));
+
+ // -f <file>
+ options.addOption(OptionBuilder
+ .hasArg()
+ .withArgName("filename")
+ .withDescription("HPL/SQL from a file")
+ .create('f'));
+
+ // -main entry_point_name
+ options.addOption(OptionBuilder
+ .hasArg()
+ .withArgName("procname")
+ .withDescription("Entry point (procedure or function name)")
+ .create("main"));
+
+ // -hiveconf x=y
+ options.addOption(OptionBuilder
+ .withValueSeparator()
+ .hasArgs(2)
+ .withArgName("property=value")
+ .withLongOpt("hiveconf")
+ .withDescription("Value for given property")
+ .create());
+
+ // Substitution option -d, --define
+ options.addOption(OptionBuilder
+ .withValueSeparator()
+ .hasArgs(2)
+ .withArgName("key=value")
+ .withLongOpt("define")
+ .withDescription("Variable subsitution e.g. -d A=B or --define A=B")
+ .create('d'));
+
+ // Substitution option --hivevar
+ options.addOption(OptionBuilder
+ .withValueSeparator()
+ .hasArgs(2)
+ .withArgName("key=value")
+ .withLongOpt("hivevar")
+ .withDescription("Variable subsitution e.g. --hivevar A=B")
+ .create());
+
+ // [-version|--version]
+ options.addOption(new Option("version", "version", false, "Print HPL/SQL version"));
+
+ // [-trace|--trace]
+ options.addOption(new Option("trace", "trace", false, "Print debug information"));
+
+ // [-offline|--offline]
+ options.addOption(new Option("offline", "offline", false, "Offline mode - skip SQL execution"));
+
+ // [-H|--help]
+ options.addOption(new Option("H", "help", false, "Print help information"));
+ }
+
+ /**
+ * Parse the command line arguments
+ */
+ public boolean parse(String[] args) {
+ try {
+ commandLine = new GnuParser().parse(options, args);
+ execString = commandLine.getOptionValue('e');
+ fileName = commandLine.getOptionValue('f');
+ main = commandLine.getOptionValue("main");
+ Properties p = commandLine.getOptionProperties("hiveconf");
+ for(String key : p.stringPropertyNames()) {
+ vars.put(key, p.getProperty(key));
+ }
+ p = commandLine.getOptionProperties("hivevar");
+ for(String key : p.stringPropertyNames()) {
+ vars.put(key, p.getProperty(key));
+ }
+ p = commandLine.getOptionProperties("define");
+ for(String key : p.stringPropertyNames()) {
+ vars.put(key, p.getProperty(key));
+ }
+ } catch (ParseException e) {
+ System.err.println(e.getMessage());
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * Get the value of execution option -e
+ */
+ public String getExecString() {
+ return execString;
+ }
+
+ /**
+ * Get the value of file option -f
+ */
+ public String getFileName() {
+ return fileName;
+ }
+
+ /**
+ * Get the value of -main option
+ */
+ public String getMain() {
+ return main;
+ }
+
+ /**
+ * Get the variables
+ */
+ public Map<String, String> getVars() {
+ return vars;
+ }
+
+ /**
+ * Test whether version option is set
+ */
+ public boolean hasVersionOption() {
+ if(commandLine.hasOption("version")) {
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Test whether debug option is set
+ */
+ public boolean hasTraceOption() {
+ if(commandLine.hasOption("trace")) {
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Test whether offline option is set
+ */
+ public boolean hasOfflineOption() {
+ if(commandLine.hasOption("offline")) {
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Test whether help option is set
+ */
+ public boolean hasHelpOption() {
+ if(commandLine.hasOption('H')) {
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Print help information
+ */
+ public void printHelp() {
+ new HelpFormatter().printHelp("hplsql", options);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Conf.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Conf.java b/hplsql/src/main/java/org/apache/hive/hplsql/Conf.java
new file mode 100644
index 0000000..88afbb5
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Conf.java
@@ -0,0 +1,175 @@
+/**
+ * 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.hive.hplsql;
+
+import java.net.URL;
+import java.util.HashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hive.hplsql.Exec.OnError;
+
+/**
+ * HPL/SQL run-time configuration
+ */
+public class Conf extends Configuration {
+
+ public static final String SITE_XML = "hplsql-site.xml";
+ public static final String DOT_HPLSQLRC = ".hplsqlrc";
+ public static final String HPLSQLRC = "hplsqlrc";
+ public static final String HPLSQL_LOCALS_SQL = "hplsql_locals.sql";
+
+ public static final String CONN_CONVERT = "hplsql.conn.convert.";
+ public static final String CONN_DEFAULT = "hplsql.conn.default";
+ public static final String DUAL_TABLE = "hplsql.dual.table";
+ public static final String INSERT_VALUES = "hplsql.insert.values";
+ public static final String ONERROR = "hplsql.onerror";
+ public static final String TEMP_TABLES = "hplsql.temp.tables";
+ public static final String TEMP_TABLES_SCHEMA = "hplsql.temp.tables.schema";
+ public static final String TEMP_TABLES_LOCATION = "hplsql.temp.tables.location";
+
+ public static final String TRUE = "true";
+ public static final String FALSE = "false";
+ public static final String YES = "yes";
+ public static final String NO = "no";
+
+ public enum InsertValues {NATIVE, SELECT};
+ public enum TempTables {NATIVE, MANAGED};
+
+ public String defaultConnection;
+
+ OnError onError = OnError.EXCEPTION;
+ InsertValues insertValues = InsertValues.NATIVE;
+ TempTables tempTables = TempTables.NATIVE;
+
+ String dualTable = "default.dual";
+
+ String tempTablesSchema = "";
+ String tempTablesLocation = "/tmp/hplsql";
+
+ HashMap<String, Boolean> connConvert = new HashMap<String, Boolean>();
+
+ /**
+ * Set an option
+ */
+ public void setOption(String key, String value) {
+ if (key.startsWith(CONN_CONVERT)) {
+ setConnectionConvert(key.substring(19), value);
+ }
+ else if (key.compareToIgnoreCase(CONN_DEFAULT) == 0) {
+ defaultConnection = value;
+ }
+ else if (key.compareToIgnoreCase(DUAL_TABLE) == 0) {
+ dualTable = value;
+ }
+ else if (key.compareToIgnoreCase(INSERT_VALUES) == 0) {
+ setInsertValues(value);
+ }
+ else if (key.compareToIgnoreCase(ONERROR) == 0) {
+ setOnError(value);
+ }
+ else if (key.compareToIgnoreCase(TEMP_TABLES) == 0) {
+ setTempTables(value);
+ }
+ else if (key.compareToIgnoreCase(TEMP_TABLES_SCHEMA) == 0) {
+ tempTablesSchema = value;
+ }
+ else if (key.compareToIgnoreCase(TEMP_TABLES_LOCATION) == 0) {
+ tempTablesLocation = value;
+ }
+ }
+
+ /**
+ * Set hplsql.insert.values option
+ */
+ private void setInsertValues(String value) {
+ if (value.compareToIgnoreCase("NATIVE") == 0) {
+ insertValues = InsertValues.NATIVE;
+ }
+ else if (value.compareToIgnoreCase("SELECT") == 0) {
+ insertValues = InsertValues.SELECT;
+ }
+ }
+
+ /**
+ * Set hplsql.temp.tables option
+ */
+ private void setTempTables(String value) {
+ if (value.compareToIgnoreCase("NATIVE") == 0) {
+ tempTables = TempTables.NATIVE;
+ }
+ else if (value.compareToIgnoreCase("MANAGED") == 0) {
+ tempTables = TempTables.MANAGED;
+ }
+ }
+
+ /**
+ * Set error handling approach
+ */
+ private void setOnError(String value) {
+ if (value.compareToIgnoreCase("EXCEPTION") == 0) {
+ onError = OnError.EXCEPTION;
+ }
+ else if (value.compareToIgnoreCase("SETERROR") == 0) {
+ onError = OnError.SETERROR;
+ }
+ if (value.compareToIgnoreCase("STOP") == 0) {
+ onError = OnError.STOP;
+ }
+ }
+
+ /**
+ * Set whether convert or not SQL for the specified connection profile
+ */
+ void setConnectionConvert(String name, String value) {
+ boolean convert = false;
+ if (value.compareToIgnoreCase(TRUE) == 0 || value.compareToIgnoreCase(YES) == 0) {
+ convert = true;
+ }
+ connConvert.put(name, convert);
+ }
+
+ /**
+ * Get whether convert or not SQL for the specified connection profile
+ */
+ boolean getConnectionConvert(String name) {
+ Boolean convert = connConvert.get(name);
+ if (convert != null) {
+ return convert.booleanValue();
+ }
+ return false;
+ }
+
+ /**
+ * Load parameters
+ */
+ public void init() {
+ addResource(SITE_XML);
+ }
+
+ /**
+ * Get the location of the configuration file
+ */
+ public String getLocation() {
+ URL url = getResource(SITE_XML);
+ if (url != null) {
+ return url.toString();
+ }
+ return "";
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
new file mode 100644
index 0000000..828fbc3
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Conn.java
@@ -0,0 +1,243 @@
+/**
+ * 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.hive.hplsql;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Stack;
+import java.sql.DriverManager;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.ResultSet;
+
+public class Conn {
+
+ public enum Type {DB2, HIVE, MYSQL, TERADATA};
+
+ HashMap<String, Stack<Connection>> connections = new HashMap<String, Stack<Connection>>();
+ HashMap<String, String> connStrings = new HashMap<String, String>();
+ HashMap<String, Type> connTypes = new HashMap<String, Type>();
+
+ HashMap<String, ArrayList<String>> connInits = new HashMap<String, ArrayList<String>>();
+ HashMap<String, ArrayList<String>> preSql = new HashMap<String, ArrayList<String>>();
+
+ Exec exec;
+ Timer timer = new Timer();
+ boolean trace = false;
+
+ Conn(Exec e) {
+ exec = e;
+ trace = exec.getTrace();
+ }
+
+ /**
+ * Execute a SQL query
+ */
+ public Query executeQuery(Query query, String connName) {
+ try {
+ Connection conn = getConnection(connName);
+ runPreSql(connName, conn);
+ Statement stmt = conn.createStatement();
+ timer.start();
+ ResultSet rs = stmt.executeQuery(query.sql);
+ timer.stop();
+ query.set(conn, stmt, rs);
+ if (trace) {
+ exec.trace(null, "Query executed successfully (" + timer.format() + ")");
+ }
+ } catch (Exception e) {
+ query.setError(e);
+ }
+ return query;
+ }
+
+ public Query executeQuery(String sql, String connName) {
+ return executeQuery(new Query(sql), connName);
+ }
+
+ /**
+ * Execute a SQL statement
+ */
+ public Query executeSql(String sql, String connName) {
+ Query query = new Query(sql);
+ try {
+ Connection conn = getConnection(connName);
+ runPreSql(connName, conn);
+ Statement stmt = conn.createStatement();
+ ResultSet rs = null;
+ if (stmt.execute(sql)) {
+ rs = stmt.getResultSet();
+ }
+ query.set(conn, stmt, rs);
+ } catch (Exception e) {
+ query.setError(e);
+ }
+ return query;
+ }
+
+ /**
+ * Close the query object
+ */
+ public void closeQuery(Query query, String connName) {
+ query.closeStatement();
+ returnConnection(connName, query.getConnection());
+ }
+
+ /**
+ * Run pre-SQL statements
+ * @throws SQLException
+ */
+ void runPreSql(String connName, Connection conn) throws SQLException {
+ ArrayList<String> sqls = preSql.get(connName);
+ if (sqls != null) {
+ Statement s = conn.createStatement();
+ for (String sql : sqls) {
+ s.execute(sql);
+ }
+ s.close();
+ preSql.remove(connName);
+ }
+ }
+
+ /**
+ * Get a connection
+ * @throws Exception
+ */
+ synchronized Connection getConnection(String connName) throws Exception {
+ Stack<Connection> connStack = connections.get(connName);
+ String connStr = connStrings.get(connName);
+ if (connStr == null) {
+ throw new Exception("Unknown connection profile: " + connName);
+ }
+ if (connStack != null && !connStack.empty()) { // Reuse an existing connection
+ return connStack.pop();
+ }
+ Connection c = openConnection(connStr);
+ ArrayList<String> sqls = connInits.get(connName); // Run initialization statements on the connection
+ if (sqls != null) {
+ Statement s = c.createStatement();
+ for (String sql : sqls) {
+ s.execute(sql);
+ }
+ s.close();
+ }
+ return c;
+ }
+
+ /**
+ * Open a new connection
+ * @throws Exception
+ */
+ Connection openConnection(String connStr) throws Exception {
+ String driver = "org.apache.hadoop.hive.jdbc.HiveDriver";
+ String url = "jdbc:hive://";
+ String usr = "";
+ String pwd = "";
+ if (connStr != null) {
+ String[] c = connStr.split(";");
+ if (c.length >= 1) {
+ driver = c[0];
+ }
+ if (c.length >= 2) {
+ url = c[1];
+ }
+ if (c.length >= 3) {
+ usr = c[2];
+ }
+ if (c.length >= 4) {
+ pwd = c[3];
+ }
+ }
+ Class.forName(driver);
+ timer.start();
+ Connection conn = DriverManager.getConnection(url, usr, pwd);
+ timer.stop();
+ if (trace) {
+ exec.trace(null, "Open connection: " + url + " (" + timer.format() + ")");
+ }
+ return conn;
+ }
+
+ /**
+ * Get the database type by profile name
+ */
+ Conn.Type getTypeByProfile(String name) {
+ return connTypes.get(name);
+ }
+
+ /**
+ * Get the database type by connection string
+ */
+ Conn.Type getType(String connStr) {
+ if (connStr.contains("hive.")) {
+ return Type.HIVE;
+ }
+ else if (connStr.contains("db2.")) {
+ return Type.DB2;
+ }
+ else if (connStr.contains("mysql.")) {
+ return Type.MYSQL;
+ }
+ else if (connStr.contains("teradata.")) {
+ return Type.TERADATA;
+ }
+ return Type.HIVE;
+ }
+
+ /**
+ * Return the connection to the pool
+ */
+ void returnConnection(String name, Connection conn) {
+ if (conn != null) {
+ connections.get(name).push(conn);
+ }
+ }
+
+ /**
+ * Add a new connection string
+ */
+ public void addConnection(String name, String connStr) {
+ connections.put(name, new Stack<Connection>());
+ connStrings.put(name, connStr);
+ connTypes.put(name, getType(connStr));
+ }
+
+ /**
+ * Add initialization statements for the specified connection
+ */
+ public void addConnectionInit(String name, String connInit) {
+ ArrayList<String> a = new ArrayList<String>();
+ String[] sa = connInit.split(";");
+ for (String s : sa) {
+ s = s.trim();
+ if (!s.isEmpty()) {
+ a.add(s);
+ }
+ }
+ connInits.put(name, a);
+ }
+
+ /**
+ * Add SQL statements to be executed before executing the next SQL statement (pre-SQL)
+ */
+ public void addPreSql(String name, ArrayList<String> sql) {
+ preSql.put(name, sql);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Converter.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Converter.java b/hplsql/src/main/java/org/apache/hive/hplsql/Converter.java
new file mode 100644
index 0000000..46f98a9
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Converter.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.hive.hplsql;
+
+/**
+ * On-the-fly SQL Converter
+ */
+public class Converter {
+
+ Exec exec;
+ boolean trace = false;
+
+ Converter(Exec e) {
+ exec = e;
+ trace = exec.getTrace();
+ }
+
+ /**
+ * Convert a data type
+ */
+ String dataType(HplsqlParser.DtypeContext type, HplsqlParser.Dtype_lenContext len) {
+ String d = null;
+ if (type.T_VARCHAR2() != null) {
+ d = "STRING";
+ }
+ else if (type.T_NUMBER() != null) {
+ d = "DECIMAL";
+ if (len != null) {
+ d += exec.getText(len);
+ }
+ }
+ if (d != null) {
+ return d;
+ }
+ else if (len != null) {
+ return exec.getText(type, type.getStart(), len.getStop());
+ }
+ return exec.getText(type, type.getStart(), type.getStop());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java b/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java
new file mode 100644
index 0000000..30b98ca
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Copy.java
@@ -0,0 +1,426 @@
+/**
+ * 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.hive.hplsql;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.List;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hive.hplsql.Var;
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.lang3.tuple.Pair;
+
+public class Copy {
+
+ Exec exec;
+ Timer timer = new Timer();
+ boolean trace = false;
+ boolean info = false;
+
+ long srcSizeInBytes = 0;
+
+ String delimiter = "\t";
+ boolean sqlInsert = false;
+ String sqlInsertName;
+ String targetConn;
+ int batchSize = 1000;
+
+ boolean overwrite = false;
+ boolean delete = false;
+ boolean ignore = false;
+
+ Copy(Exec e) {
+ exec = e;
+ trace = exec.getTrace();
+ info = exec.getInfo();
+ }
+
+ /**
+ * Run COPY command
+ */
+ Integer run(HplsqlParser.Copy_stmtContext ctx) {
+ trace(ctx, "COPY");
+ initOptions(ctx);
+ StringBuilder sql = new StringBuilder();
+ String conn = null;
+ if (ctx.table_name() != null) {
+ String table = evalPop(ctx.table_name()).toString();
+ conn = exec.getObjectConnection(ctx.table_name().getText());
+ sql.append("SELECT * FROM ");
+ sql.append(table);
+ }
+ else {
+ sql.append(evalPop(ctx.select_stmt()).toString());
+ conn = exec.getStatementConnection();
+ if (trace) {
+ trace(ctx, "Statement:\n" + sql);
+ }
+ }
+ Query query = exec.executeQuery(ctx, sql.toString(), conn);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ try {
+ if (targetConn != null) {
+ copyToTable(ctx, query);
+ }
+ else {
+ copyToFile(ctx, query);
+ }
+ }
+ catch (Exception e) {
+ exec.signal(e);
+ return 1;
+ }
+ finally {
+ exec.closeQuery(query, conn);
+ }
+ return 0;
+ }
+
+ /**
+ * Copy the query results to another table
+ * @throws Exception
+ */
+ void copyToTable(HplsqlParser.Copy_stmtContext ctx, Query query) throws Exception {
+ ResultSet rs = query.getResultSet();
+ if (rs == null) {
+ return;
+ }
+ ResultSetMetaData rm = rs.getMetaData();
+ int cols = rm.getColumnCount();
+ int rows = 0;
+ if (trace) {
+ trace(ctx, "SELECT executed: " + cols + " columns");
+ }
+ Connection conn = exec.getConnection(targetConn);
+ StringBuilder sql = new StringBuilder();
+ sql.append("INSERT INTO " + sqlInsertName + " VALUES (");
+ for (int i = 0; i < cols; i++) {
+ sql.append("?");
+ if (i + 1 < cols) {
+ sql.append(",");
+ }
+ }
+ sql.append(")");
+ PreparedStatement ps = conn.prepareStatement(sql.toString());
+ long start = timer.start();
+ long prev = start;
+ boolean batchOpen = false;
+ while (rs.next()) {
+ for (int i = 1; i <= cols; i++) {
+ ps.setObject(i, rs.getObject(i));
+ }
+ rows++;
+ if (batchSize > 1) {
+ ps.addBatch();
+ batchOpen = true;
+ if (rows % batchSize == 0) {
+ ps.executeBatch();
+ batchOpen = false;
+ }
+ }
+ else {
+ ps.executeUpdate();
+ }
+ if (trace && rows % 100 == 0) {
+ long cur = timer.current();
+ if (cur - prev > 10000) {
+ trace(ctx, "Copying rows: " + rows + " (" + rows/((cur - start)/1000) + " rows/sec)");
+ prev = cur;
+ }
+ }
+ }
+ if (batchOpen) {
+ ps.executeBatch();
+ }
+ ps.close();
+ exec.returnConnection(targetConn, conn);
+ exec.setRowCount(rows);
+ long elapsed = timer.stop();
+ if (trace) {
+ trace(ctx, "COPY completed: " + rows + " row(s), " + timer.format() + ", " + rows/(elapsed/1000) + " rows/sec");
+ }
+ }
+
+ /**
+ * Copy the query results to a file
+ * @throws Exception
+ */
+ void copyToFile(HplsqlParser.Copy_stmtContext ctx, Query query) throws Exception {
+ ResultSet rs = query.getResultSet();
+ if (rs == null) {
+ return;
+ }
+ ResultSetMetaData rm = rs.getMetaData();
+ String filename = null;
+ if (ctx.copy_target().expr() != null) {
+ filename = evalPop(ctx.copy_target().expr()).toString();
+ }
+ else {
+ filename = ctx.copy_target().getText();
+ }
+ byte[] del = delimiter.getBytes();
+ byte[] rowdel = "\n".getBytes();
+ byte[] nullstr = "NULL".getBytes();
+ int cols = rm.getColumnCount();
+ int rows = 0;
+ if (trace) {
+ trace(ctx, "SELECT executed: " + cols + " columns, output file: " + filename);
+ }
+ java.io.File file = new java.io.File(filename);
+ FileOutputStream out = null;
+ try {
+ if (!file.exists()) {
+ file.createNewFile();
+ }
+ out = new FileOutputStream(file, false /*append*/);
+ String col;
+ String sql = "";
+ if (sqlInsert) {
+ sql = "INSERT INTO " + sqlInsertName + " VALUES (";
+ rowdel = ");\n".getBytes();
+ }
+ while (rs.next()) {
+ if (sqlInsert) {
+ out.write(sql.getBytes());
+ }
+ for (int i = 1; i <= cols; i++) {
+ if (i > 1) {
+ out.write(del);
+ }
+ col = rs.getString(i);
+ if (col != null) {
+ if (sqlInsert) {
+ col = Utils.quoteString(col);
+ }
+ out.write(col.getBytes());
+ }
+ else if (sqlInsert) {
+ out.write(nullstr);
+ }
+ }
+ out.write(rowdel);
+ rows++;
+ }
+ exec.setRowCount(rows);
+ }
+ finally {
+ if (out != null) {
+ out.close();
+ }
+ }
+ if (trace) {
+ trace(ctx, "COPY rows: " + rows);
+ }
+ }
+
+ /**
+ * Run COPY FROM LOCAL statement
+ */
+ public Integer runFromLocal(HplsqlParser.Copy_from_local_stmtContext ctx) {
+ trace(ctx, "COPY FROM LOCAL");
+ initFileOptions(ctx.copy_file_option());
+ HashMap<String, Pair<String, Long>> src = new HashMap<String, Pair<String, Long>>();
+ int cnt = ctx.copy_source().size();
+ for (int i = 0; i < cnt; i++) {
+ createLocalFileList(src, evalPop(ctx.copy_source(i)).toString(), null);
+ }
+ String dest = evalPop(ctx.copy_target()).toString();
+ if (info) {
+ info(ctx, "Files to copy: " + src.size() + " (" + Utils.formatSizeInBytes(srcSizeInBytes) + ")");
+ }
+ timer.start();
+ File file = new File();
+ FileSystem fs = null;
+ int succeed = 0;
+ int failed = 0;
+ long copiedSize = 0;
+ try {
+ fs = file.createFs();
+ boolean multi = false;
+ if (src.size() > 1) {
+ multi = true;
+ }
+ for (Map.Entry<String, Pair<String, Long>> i : src.entrySet()) {
+ try {
+ Path s = new Path(i.getKey());
+ Path d = null;
+ if (multi) {
+ String relativePath = i.getValue().getLeft();
+ if (relativePath == null) {
+ d = new Path(dest, s.getName());
+ }
+ else {
+ d = new Path(dest, relativePath + java.io.File.separator + s.getName());
+ }
+ }
+ else {
+ d = new Path(dest);
+ }
+ fs.copyFromLocalFile(delete, overwrite, s, d);
+ succeed++;
+ long size = i.getValue().getRight();
+ copiedSize += size;
+ if (info) {
+ info(ctx, "Copied: " + file.resolvePath(d) + " (" + Utils.formatSizeInBytes(size) + ")");
+ }
+ }
+ catch(IOException e) {
+ failed++;
+ if (!ignore) {
+ throw e;
+ }
+ }
+ }
+ }
+ catch(IOException e) {
+ exec.signal(e);
+ exec.setHostCode(1);
+ return 1;
+ }
+ finally {
+ long elapsed = timer.stop();
+ if (info) {
+ info(ctx, "COPY completed: " + succeed + " succeed, " + failed + " failed, " +
+ timer.format() + ", " + Utils.formatSizeInBytes(copiedSize) + ", " +
+ Utils.formatBytesPerSec(copiedSize, elapsed));
+ }
+ if (failed == 0) {
+ exec.setHostCode(0);
+ }
+ else {
+ exec.setHostCode(1);
+ }
+ file.close();
+ }
+ return 0;
+ }
+
+ /**
+ * Create the list of local files for the specified path (including subdirectories)
+ */
+ void createLocalFileList(HashMap<String, Pair<String, Long>> list, String path, String relativePath) {
+ java.io.File file = new java.io.File(path);
+ if (file.exists()) {
+ if (file.isDirectory()) {
+ for (java.io.File i : file.listFiles()) {
+ if (i.isDirectory()) {
+ String rel = null;
+ if (relativePath == null) {
+ rel = i.getName();
+ }
+ else {
+ rel = relativePath + java.io.File.separator + i.getName();
+ }
+ createLocalFileList(list, i.getAbsolutePath(), rel);
+ }
+ else {
+ long size = i.length();
+ list.put(i.getAbsolutePath(), Pair.of(relativePath, size));
+ srcSizeInBytes += size;
+ }
+ }
+ }
+ else {
+ long size = file.length();
+ list.put(file.getAbsolutePath(), Pair.of(relativePath, size));
+ srcSizeInBytes += size;
+ }
+ }
+ }
+
+ /**
+ * Initialize COPY command options
+ */
+ void initOptions(HplsqlParser.Copy_stmtContext ctx) {
+ int cnt = ctx.copy_option().size();
+ for (int i = 0; i < cnt; i++) {
+ HplsqlParser.Copy_optionContext option = ctx.copy_option(i);
+ if (option.T_DELIMITER() != null) {
+ delimiter = StringEscapeUtils.unescapeJava(evalPop(option.expr()).toString());
+ }
+ else if (option.T_SQLINSERT() != null) {
+ sqlInsert = true;
+ delimiter = ", ";
+ if (option.ident() != null) {
+ sqlInsertName = option.ident().getText();
+ }
+ }
+ else if (option.T_AT() != null) {
+ targetConn = option.ident().getText();
+ sqlInsertName = ctx.copy_target().ident().getText();
+ }
+ else if (option.T_BATCHSIZE() != null) {
+ batchSize = evalPop(option.expr()).intValue();
+ }
+ }
+ }
+
+ /**
+ * Initialize COPY FILE options
+ */
+ void initFileOptions(List<HplsqlParser.Copy_file_optionContext> options) {
+ srcSizeInBytes = 0;
+ for (HplsqlParser.Copy_file_optionContext i : options) {
+ if (i.T_OVERWRITE() != null) {
+ overwrite = true;
+ }
+ else if (i.T_DELETE() != null) {
+ delete = true;
+ }
+ else if (i.T_IGNORE() != null) {
+ ignore = true;
+ }
+ }
+ }
+
+ /**
+ * Evaluate the expression and pop value from the stack
+ */
+ Var evalPop(ParserRuleContext ctx) {
+ exec.visit(ctx);
+ if (!exec.stack.isEmpty()) {
+ return exec.stackPop();
+ }
+ return Var.Empty;
+ }
+
+ /**
+ * Trace and information
+ */
+ public void trace(ParserRuleContext ctx, String message) {
+ exec.trace(ctx, message);
+ }
+
+ public void info(ParserRuleContext ctx, String message) {
+ exec.info(ctx, message);
+ }
+}
[24/25] hive git commit: HIVE-11102 : ReaderImpl:
getColumnIndicesFromNames does not work for some cases (Sergey Shelukhin,
reviewed by Prasanth Jayachandran)
Posted by se...@apache.org.
HIVE-11102 : ReaderImpl: getColumnIndicesFromNames does not work for some cases (Sergey Shelukhin, reviewed by Prasanth Jayachandran)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/023c06c1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/023c06c1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/023c06c1
Branch: refs/heads/llap
Commit: 023c06c17d7e03ba0458ed2bac6b475102150d77
Parents: 6eaa32c
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Jul 1 16:59:21 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Jul 1 16:59:21 2015 -0700
----------------------------------------------------------------------
ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java | 7 +++++++
1 file changed, 7 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/023c06c1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
index bbc4654..3c0de3c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/ReaderImpl.java
@@ -642,6 +642,13 @@ public class ReaderImpl implements Reader {
for (String colName : colNames) {
if (fieldNames.contains(colName)) {
fieldIdx = fieldNames.indexOf(colName);
+ } else {
+ String s = "Cannot find field for: " + colName + " in ";
+ for (String fn : fieldNames) {
+ s += fn + ", ";
+ }
+ LOG.warn(s);
+ continue;
}
// a single field may span multiple columns. find start and end column
[04/25] hive git commit: HIVE-11100: Beeline should escape semi-colon
in queries (Chaoyu via Xuefu)
Posted by se...@apache.org.
HIVE-11100: Beeline should escape semi-colon in queries (Chaoyu via Xuefu)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/3991dba3
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/3991dba3
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/3991dba3
Branch: refs/heads/llap
Commit: 3991dba30c5068cac296f32e24e97cf87efa266c
Parents: 9c64f93
Author: xzhang <xz...@xzdt>
Authored: Tue Jun 30 10:43:10 2015 -0700
Committer: xzhang <xz...@xzdt>
Committed: Tue Jun 30 10:43:10 2015 -0700
----------------------------------------------------------------------
.../java/org/apache/hive/beeline/Commands.java | 22 +++--
.../hive/beeline/TestBeeLineWithArgs.java | 87 ++++++++++++++++++++
2 files changed, 103 insertions(+), 6 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/3991dba3/beeline/src/java/org/apache/hive/beeline/Commands.java
----------------------------------------------------------------------
diff --git a/beeline/src/java/org/apache/hive/beeline/Commands.java b/beeline/src/java/org/apache/hive/beeline/Commands.java
index 3ff1ef0..3cdcfb8 100644
--- a/beeline/src/java/org/apache/hive/beeline/Commands.java
+++ b/beeline/src/java/org/apache/hive/beeline/Commands.java
@@ -43,6 +43,7 @@ import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.sql.SQLWarning;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator;
import java.util.LinkedList;
@@ -797,15 +798,24 @@ public class Commands {
}
line = line.trim();
- String[] cmds;
+ List<String> cmdList = new ArrayList<String>();
if (entireLineAsCommand) {
- cmds = new String[1];
- cmds[0] = line;
+ cmdList.add(line);
} else {
- cmds = line.split(";");
+ StringBuffer command = new StringBuffer();
+ for (String cmdpart: line.split(";")) {
+ if (cmdpart.endsWith("\\")) {
+ command.append(cmdpart.substring(0, cmdpart.length() -1)).append(";");
+ continue;
+ } else {
+ command.append(cmdpart);
+ }
+ cmdList.add(command.toString());
+ command.setLength(0);
+ }
}
- for (int i = 0; i < cmds.length; i++) {
- String sql = cmds[i].trim();
+ for (int i = 0; i < cmdList.size(); i++) {
+ String sql = cmdList.get(i).trim();
if (sql.length() != 0) {
if (beeLine.isComment(sql)) {
//skip this and rest cmds in the line
http://git-wip-us.apache.org/repos/asf/hive/blob/3991dba3/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
index f0795d2..19ee621 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/beeline/TestBeeLineWithArgs.java
@@ -194,6 +194,31 @@ public class TestBeeLineWithArgs {
}
/**
+ * Attempt to execute the enclosed query with the -e option to BeeLine
+ * Test for presence of an expected pattern
+ * in the output (stdout or stderr), fail if not found
+ * Print PASSED or FAILED
+ * @param expectedPattern Text to look for in command output/error
+ * @param shouldMatch true if the pattern should be found, false if it should not
+ * @throws Exception on command execution error
+ */
+ private void testCommandEnclosedQuery(String enclosedQuery, String expectedPattern,
+ boolean shouldMatch, List<String> argList) throws Throwable {
+
+ List<String> copy = new ArrayList<String>(argList);
+ copy.add("-e");
+ copy.add(enclosedQuery);
+
+ String output = testCommandLineScript(copy, null);
+ boolean matches = output.contains(expectedPattern);
+ if (shouldMatch != matches) {
+ //failed
+ fail("Output" + output + " should" + (shouldMatch ? "" : " not") +
+ " contain " + expectedPattern);
+ }
+ }
+
+ /**
* Test that BeeLine will read comment lines that start with whitespace
* @throws Throwable
*/
@@ -652,4 +677,66 @@ public class TestBeeLineWithArgs {
final String EXPECTED_PATTERN = "Parsing command";
testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, false, getBaseArgs(miniHS2.getBaseJdbcURL()));
}
+
+ @Test
+ public void testMultiCommandsInOneline() throws Throwable {
+ final String SCRIPT_TEXT = "drop table if exists multiCmdTbl;create table multiCmdTbl "
+ +"(key int);show tables; --multicommands in one line";
+ final String EXPECTED_PATTERN = " multicmdtbl ";
+ List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+ testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+
+ final String SCRIPT_TEXT_DROP = "drop table multiCmdTbl;show tables;";
+ testScriptFile(SCRIPT_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+ }
+
+ @Test
+ public void testMultiCommandsInOneEnclosedQuery() throws Throwable {
+ final String QUERY_TEXT = "drop table if exists multiCmdTbl;create table multiCmdTbl "
+ +"(key int);show tables; --multicommands in one line";
+ final String EXPECTED_PATTERN = " multicmdtbl ";
+ List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+ testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList);
+
+ final String QUERY_TEXT_DROP = "drop table multiCmdTbl;show tables;";
+ testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+ }
+
+ @Test
+ public void testOneCommandInMultiLines() throws Throwable {
+ final String SCRIPT_TEXT = "drop table if exists multiCmdTbl;create table \nmultiCmdTbl "
+ + "(key int);show tables; --one command in multiple lines";
+ final String EXPECTED_PATTERN = " multicmdtbl ";
+ List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+ testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+
+ final String SCRIPT_TEXT_DROP = "drop table\nmultiCmdTbl;show tables;";
+ testScriptFile(SCRIPT_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+ }
+
+ @Test
+ public void testEscapeSemiColonInQueries() throws Throwable {
+ final String SCRIPT_TEXT = "drop table if exists multiCmdTbl;create table multiCmdTbl "
+ + "(key int, value string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\\;' LINES "
+ + " TERMINATED BY '\\n';show tables; --one command in multiple lines";
+ final String EXPECTED_PATTERN = " multicmdtbl ";
+ List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+ testScriptFile(SCRIPT_TEXT, EXPECTED_PATTERN, true, argList);
+
+ final String SCRIPT_TEXT_DROP = "drop table\nmultiCmdTbl;show tables;";
+ testScriptFile(SCRIPT_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+ }
+
+ @Test
+ public void testEscapeSemiColonInEnclosedQuery() throws Throwable {
+ final String QUERY_TEXT = "drop table if exists multiCmdTbl;create table multiCmdTbl "
+ + "(key int, value string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\\;' LINES "
+ + " TERMINATED BY '\\n';show tables;";
+ final String EXPECTED_PATTERN = " multicmdtbl ";
+ List<String> argList = getBaseArgs(miniHS2.getBaseJdbcURL());
+ testCommandEnclosedQuery(QUERY_TEXT, EXPECTED_PATTERN, true, argList);
+
+ final String QUERY_TEXT_DROP = "drop table multiCmdTbl;show tables;";
+ testCommandEnclosedQuery(QUERY_TEXT_DROP, EXPECTED_PATTERN, false, argList);
+ }
}
[16/25] hive git commit: HIVE-11086: Remove use of ErrorMsg in Orc's
RunLengthIntegerReaderV2 (Owen O'Malley via Prasanth Jayachandran)
Posted by se...@apache.org.
HIVE-11086: Remove use of ErrorMsg in Orc's RunLengthIntegerReaderV2 (Owen O'Malley via Prasanth Jayachandran)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/c6d3e68a
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/c6d3e68a
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/c6d3e68a
Branch: refs/heads/llap
Commit: c6d3e68a7eed553f9a231f789866d3241c487ae8
Parents: 20034d3
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Wed Jul 1 10:09:14 2015 -0700
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Wed Jul 1 10:09:14 2015 -0700
----------------------------------------------------------------------
ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java | 3 +--
.../hadoop/hive/ql/io/orc/RunLengthIntegerReaderV2.java | 8 +++-----
2 files changed, 4 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/c6d3e68a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index f012b72..fbdd66a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -490,8 +490,7 @@ public enum ErrorMsg {
"Stats type {0} is missing from stats aggregator. If you don't want the query " +
"to fail because of this, set hive.stats.atomic=false", true),
STATS_SKIPPING_BY_ERROR(30017, "Skipping stats aggregation by error {0}", true),
- ORC_CORRUPTED_READ(30018, "Corruption in ORC data encountered. To skip reading corrupted "
- + "data, set " + HiveConf.ConfVars.HIVE_ORC_SKIP_CORRUPT_DATA + " to true"),
+
INVALID_FILE_FORMAT_IN_LOAD(30019, "The file that you are trying to load does not match the" +
" file format of the destination table.")
http://git-wip-us.apache.org/repos/asf/hive/blob/c6d3e68a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RunLengthIntegerReaderV2.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RunLengthIntegerReaderV2.java b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RunLengthIntegerReaderV2.java
index f406e17..ec46595 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RunLengthIntegerReaderV2.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RunLengthIntegerReaderV2.java
@@ -20,10 +20,6 @@ package org.apache.hadoop.hive.ql.io.orc;
import java.io.EOFException;
import java.io.IOException;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
import org.apache.hadoop.hive.ql.io.orc.RunLengthIntegerWriterV2.EncodingType;
@@ -174,7 +170,9 @@ class RunLengthIntegerReaderV2 implements IntegerReader {
long[] unpackedPatch = new long[pl];
if ((pw + pgw) > 64 && !skipCorrupt) {
- throw new IOException(ErrorMsg.ORC_CORRUPTED_READ.getMsg());
+ throw new IOException("Corruption in ORC data encountered. To skip" +
+ " reading corrupted data, set hive.exec.orc.skip.corrupt.data to" +
+ " true");
}
int bitSize = utils.getClosestFixedBits(pw + pgw);
utils.readInts(unpackedPatch, 0, pl, bitSize, input);
[17/25] hive git commit: HIVE-11151 : Calcite transitive predicate
inference rule should not transitively add not null filter on non-nullable
input (Ashutosh Chauhan via Jesus Camacho Rodriguez)
Posted by se...@apache.org.
HIVE-11151 : Calcite transitive predicate inference rule should not transitively add not null filter on non-nullable input (Ashutosh Chauhan via Jesus Camacho Rodriguez)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/b5fb31cd
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/b5fb31cd
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/b5fb31cd
Branch: refs/heads/llap
Commit: b5fb31cde21887dea893b0bd410a6f49f80acb13
Parents: c6d3e68
Author: Ashutosh Chauhan <ha...@apache.org>
Authored: Tue Jun 30 15:20:13 2015 -0700
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Wed Jul 1 13:53:52 2015 -0700
----------------------------------------------------------------------
.../HiveJoinPushTransitivePredicatesRule.java | 139 +++++++++++++++++++
.../hadoop/hive/ql/parse/CalcitePlanner.java | 5 +-
2 files changed, 142 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/b5fb31cd/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
new file mode 100644
index 0000000..29deed9
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveJoinPushTransitivePredicatesRule.java
@@ -0,0 +1,139 @@
+/*
+ * 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.hadoop.hive.ql.optimizer.calcite.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDFOPNotNull;
+import org.apache.hive.common.util.AnnotationUtils;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Planner rule that infers predicates from on a
+ * {@link org.apache.calcite.rel.core.Join} and creates
+ * {@link org.apache.calcite.rel.core.Filter}s if those predicates can be pushed
+ * to its inputs.
+ *
+ * <p>Uses {@link org.apache.calcite.rel.metadata.RelMdPredicates} to infer
+ * the predicates,
+ * returns them in a {@link org.apache.calcite.plan.RelOptPredicateList}
+ * and applies them appropriately.
+ */
+public class HiveJoinPushTransitivePredicatesRule extends RelOptRule {
+ private final RelFactories.FilterFactory filterFactory;
+
+ /** The singleton. */
+ public static final HiveJoinPushTransitivePredicatesRule INSTANCE =
+ new HiveJoinPushTransitivePredicatesRule(Join.class,
+ RelFactories.DEFAULT_FILTER_FACTORY);
+
+ public HiveJoinPushTransitivePredicatesRule(Class<? extends Join> clazz,
+ RelFactories.FilterFactory filterFactory) {
+ super(operand(clazz, operand(RelNode.class, any()),
+ operand(RelNode.class, any())));
+ this.filterFactory = filterFactory;
+ }
+
+ @Override public void onMatch(RelOptRuleCall call) {
+ Join join = call.rel(0);
+ RelOptPredicateList preds = RelMetadataQuery.getPulledUpPredicates(join);
+
+ RexBuilder rB = join.getCluster().getRexBuilder();
+ RelNode lChild = call.rel(1);
+ RelNode rChild = call.rel(2);
+
+ List<RexNode> leftPreds = getValidPreds(preds.leftInferredPredicates, lChild.getRowType().getFieldList());
+ List<RexNode> rightPreds = getValidPreds(preds.rightInferredPredicates, rChild.getRowType().getFieldList());
+
+ if (leftPreds.isEmpty() && rightPreds.isEmpty()) {
+ return;
+ }
+
+ if (leftPreds.size() > 0) {
+ RelNode curr = lChild;
+ lChild = filterFactory.createFilter(lChild, RexUtil.composeConjunction(rB, leftPreds, false));
+ call.getPlanner().onCopy(curr, lChild);
+ }
+
+ if (rightPreds.size() > 0) {
+ RelNode curr = rChild;
+ rChild = filterFactory.createFilter(rChild, RexUtil.composeConjunction(rB, rightPreds, false));
+ call.getPlanner().onCopy(curr, rChild);
+ }
+
+ RelNode newRel = join.copy(join.getTraitSet(), join.getCondition(),
+ lChild, rChild, join.getJoinType(), join.isSemiJoinDone());
+ call.getPlanner().onCopy(join, newRel);
+
+ call.transformTo(newRel);
+ }
+
+ private ImmutableList<RexNode> getValidPreds (List<RexNode> rexs, List<RelDataTypeField> types) {
+ InputRefValidator validator = new InputRefValidator(types);
+ List<RexNode> valids = new ArrayList<RexNode>(rexs.size());
+ for (RexNode rex : rexs) {
+ try {
+ rex.accept(validator);
+ valids.add(rex);
+ } catch (Util.FoundOne e) {
+ Util.swallow(e, null);
+ }
+ }
+ return ImmutableList.copyOf(valids);
+ }
+
+ private static class InputRefValidator extends RexVisitorImpl<Void> {
+
+ private final List<RelDataTypeField> types;
+ protected InputRefValidator(List<RelDataTypeField> types) {
+ super(true);
+ this.types = types;
+ }
+
+ @Override
+ public Void visitCall(RexCall call) {
+
+ if(AnnotationUtils.getAnnotation(GenericUDFOPNotNull.class, Description.class).name().equals(call.getOperator().getName())) {
+ if(call.getOperands().get(0) instanceof RexInputRef &&
+ !types.get(((RexInputRef)call.getOperands().get(0)).getIndex()).getType().isNullable()) {
+ // No need to add not null filter for a constant.
+ throw new Util.FoundOne(call);
+ }
+ }
+ return super.visitCall(call);
+ }
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/hive/blob/b5fb31cd/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index a73e24e..7fd8c85 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -140,6 +140,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveFilterSetOpTranspos
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveInsertExchange4JoinRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinAddNotNullRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinProjectTransposeRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinPushTransitivePredicatesRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveJoinToMultiJoinRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePartitionPruneRule;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HivePreFilteringRule;
@@ -265,7 +266,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
// here with SemiJoins not all tables are costed in CBO, so their
// PartitionList is not evaluated until the run phase.
getMetaData(getQB());
-
+
disableJoinMerge = false;
sinkOp = genPlan(getQB());
LOG.info("CBO Succeeded; optimized logical plan.");
@@ -965,7 +966,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
HiveFilter.DEFAULT_FILTER_FACTORY, Aggregate.class));
// 4. Transitive inference & Partition Pruning
- basePlan = hepPlan(basePlan, false, mdProvider, new JoinPushTransitivePredicatesRule(
+ basePlan = hepPlan(basePlan, false, mdProvider, new HiveJoinPushTransitivePredicatesRule(
Join.class, HiveFilter.DEFAULT_FILTER_FACTORY),
new HivePartitionPruneRule(conf));
[20/25] hive git commit: HIVE-11152 : Swapping join inputs in
ASTConverter (Jesus Camacho Rodriguez via John Pullokkaran)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out b/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
index 7089004..3b2c80e 100644
--- a/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/spark/mapjoin_mapjoin.q.out
@@ -65,25 +65,25 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 1
+ Map 2
Map Operator Tree:
TableScan
- alias: src
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ alias: src1
+ Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: value is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: value (type: string)
+ expressions: key (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
0 _col0 (type: string)
- 1 _col1 (type: string)
- Position of Big Table: 1
+ 1 _col0 (type: string)
+ Position of Big Table: 0
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -91,7 +91,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src
+ base file name: src1
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -101,14 +101,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -121,37 +121,37 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src
- name: default.src
+ name: default.src1
+ name: default.src1
Truncated Path -> Alias:
- /src [src]
+ /src1 [src1]
Map 3
Map Operator Tree:
TableScan
- alias: src1
- Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ alias: src
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: key is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ predicate: value is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: string)
+ expressions: value (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
- 0 _col0 (type: string)
+ 0 _col1 (type: string)
1 _col0 (type: string)
Position of Big Table: 0
Local Work:
@@ -161,7 +161,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src1
+ base file name: src
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -171,14 +171,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -191,26 +191,26 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src1
- name: default.src1
+ name: default.src
+ name: default.src
Truncated Path -> Alias:
- /src1 [src1]
+ /src [src]
Stage: Stage-1
Spark
#### A masked pattern was here ####
Vertices:
- Map 2
+ Map 1
Map Operator Tree:
TableScan
alias: srcpart
@@ -232,45 +232,41 @@ STAGE PLANS:
1 _col0 (type: string)
outputColumnNames: _col0, _col1
input vertices:
- 1 Map 3
+ 1 Map 2
Position of Big Table: 0
Statistics: Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0
input vertices:
- 0 Map 1
- Position of Big Table: 1
+ 1 Map 3
+ Position of Big Table: 0
Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- GlobalTableId: 0
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
#### A masked pattern was here ####
- NumFilesPerFileSink: 1
- Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
#### A masked pattern was here ####
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- properties:
- columns _col0
- columns.types string
- escape.delim \
- hive.serialization.extend.additional.nesting.levels true
- serialization.format 1
- serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- TotalFiles: 1
- GatherStats: false
- MultiFileSpray: false
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ properties:
+ columns _col0
+ columns.types string
+ escape.delim \
+ hive.serialization.extend.additional.nesting.levels true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -488,39 +484,39 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 1
+ Map 2
Map Operator Tree:
TableScan
- alias: src
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ alias: src1
+ Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (value > 'val_450') (type: boolean)
- Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: value (type: string)
+ expressions: key (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
0 _col0 (type: string)
- 1 _col1 (type: string)
+ 1 _col0 (type: string)
Local Work:
Map Reduce Local Work
Map 3
Map Operator Tree:
TableScan
- alias: src1
- Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ alias: src
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: key is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ predicate: (value > 'val_450') (type: boolean)
+ Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: key (type: string)
+ expressions: value (type: string)
outputColumnNames: _col0
- Statistics: Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 166 Data size: 1763 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
- 0 _col0 (type: string)
+ 0 _col1 (type: string)
1 _col0 (type: string)
Local Work:
Map Reduce Local Work
@@ -529,7 +525,7 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 2
+ Map 1
Map Operator Tree:
TableScan
alias: srcpart
@@ -549,29 +545,25 @@ STAGE PLANS:
1 _col0 (type: string)
outputColumnNames: _col0, _col1
input vertices:
- 1 Map 3
+ 1 Map 2
Statistics: Num rows: 183 Data size: 1951 Basic stats: COMPLETE Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col1
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0
input vertices:
- 0 Map 1
+ 1 Map 3
Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col1 (type: string)
- outputColumnNames: _col0
+ File Output Operator
+ compressed: false
Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 201 Data size: 2146 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Local Work:
Map Reduce Local Work
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
index c1bfaff..2147984 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_1.q.out
@@ -6427,77 +6427,74 @@ POSTHOOK: type: QUERY
Plan optimized by CBO.
Vertex dependency in root stage
-Map 3 <- Map 4 (BROADCAST_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+Map 1 <- Map 3 (BROADCAST_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
Reducer 2
- File Output Operator [FS_20]
+ File Output Operator [FS_19]
compressed:true
Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
- Select Operator [SEL_19]
- outputColumnNames:["_col0"]
- Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
- Merge Join Operator [MERGEJOIN_30]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col1 (type: string)","0":"_col0 (type: string)"}
- | outputColumnNames:["_col1"]
- | Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 1 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_15]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | Select Operator [SEL_2]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | Filter Operator [FIL_26]
- | predicate:(value > 'val_450') (type: boolean)
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | TableScan [TS_0]
- | alias:src
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 3 [SIMPLE_EDGE]
- Reduce Output Operator [RS_17]
- key expressions:_col1 (type: string)
- Map-reduce partition columns:_col1 (type: string)
- sort order:+
- Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions:_col0 (type: string)
- Map Join Operator [MAPJOIN_29]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"Map 3":"_col0 (type: string)","Map 4":"_col0 (type: string)"}
- | outputColumnNames:["_col0","_col1"]
- | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 4 [BROADCAST_EDGE]
- | Reduce Output Operator [RS_11]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | Select Operator [SEL_7]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | Filter Operator [FIL_28]
- | predicate:key is not null (type: boolean)
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | TableScan [TS_6]
- | alias:src1
- | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Select Operator [SEL_5]
- outputColumnNames:["_col0","_col1"]
- Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator [FIL_27]
- predicate:(((value > 'val_450') and key is not null) and value is not null) (type: boolean)
- Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
- TableScan [TS_3]
- alias:srcpart
- Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+ Merge Join Operator [MERGEJOIN_29]
+ | condition map:[{"":"Inner Join 0 to 1"}]
+ | keys:{"1":"_col0 (type: string)","0":"_col1 (type: string)"}
+ | outputColumnNames:["_col0"]
+ | Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
+ |<-Map 1 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_14]
+ | key expressions:_col1 (type: string)
+ | Map-reduce partition columns:_col1 (type: string)
+ | sort order:+
+ | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
+ | value expressions:_col0 (type: string)
+ | Map Join Operator [MAPJOIN_28]
+ | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | keys:{"Map 1":"_col0 (type: string)","Map 3":"_col0 (type: string)"}
+ | | outputColumnNames:["_col0","_col1"]
+ | | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
+ | |<-Map 3 [BROADCAST_EDGE]
+ | | Reduce Output Operator [RS_11]
+ | | key expressions:_col0 (type: string)
+ | | Map-reduce partition columns:_col0 (type: string)
+ | | sort order:+
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | Select Operator [SEL_4]
+ | | outputColumnNames:["_col0"]
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | Filter Operator [FIL_26]
+ | | predicate:key is not null (type: boolean)
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | TableScan [TS_3]
+ | | alias:src1
+ | | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+ | |<-Select Operator [SEL_2]
+ | outputColumnNames:["_col0","_col1"]
+ | Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
+ | Filter Operator [FIL_25]
+ | predicate:(((value > 'val_450') and key is not null) and value is not null) (type: boolean)
+ | Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
+ | TableScan [TS_0]
+ | alias:srcpart
+ | Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+ |<-Map 4 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_16]
+ key expressions:_col0 (type: string)
+ Map-reduce partition columns:_col0 (type: string)
+ sort order:+
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator [SEL_7]
+ outputColumnNames:["_col0"]
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator [FIL_27]
+ predicate:(value > 'val_450') (type: boolean)
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ TableScan [TS_5]
+ alias:src
+ Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
PREHOOK: query: explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'
PREHOOK: type: QUERY
@@ -6506,77 +6503,74 @@ POSTHOOK: type: QUERY
Plan optimized by CBO.
Vertex dependency in root stage
-Map 3 <- Map 4 (BROADCAST_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+Map 1 <- Map 3 (BROADCAST_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
Reducer 2
- File Output Operator [FS_20]
+ File Output Operator [FS_19]
compressed:true
Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
- Select Operator [SEL_19]
- outputColumnNames:["_col0"]
- Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
- Merge Join Operator [MERGEJOIN_30]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col1 (type: string)","0":"_col0 (type: string)"}
- | outputColumnNames:["_col1"]
- | Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 1 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_15]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | Select Operator [SEL_2]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | Filter Operator [FIL_26]
- | predicate:(value > 'val_450') (type: boolean)
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | TableScan [TS_0]
- | alias:src
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 3 [SIMPLE_EDGE]
- Reduce Output Operator [RS_17]
- key expressions:_col1 (type: string)
- Map-reduce partition columns:_col1 (type: string)
- sort order:+
- Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions:_col0 (type: string)
- Map Join Operator [MAPJOIN_29]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"Map 3":"_col0 (type: string)","Map 4":"_col0 (type: string)"}
- | outputColumnNames:["_col0","_col1"]
- | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 4 [BROADCAST_EDGE]
- | Reduce Output Operator [RS_11]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | Select Operator [SEL_7]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | Filter Operator [FIL_28]
- | predicate:key is not null (type: boolean)
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | TableScan [TS_6]
- | alias:src1
- | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Select Operator [SEL_5]
- outputColumnNames:["_col0","_col1"]
- Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator [FIL_27]
- predicate:(((value > 'val_450') and key is not null) and value is not null) (type: boolean)
- Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
- TableScan [TS_3]
- alias:srcpart
- Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+ Merge Join Operator [MERGEJOIN_29]
+ | condition map:[{"":"Inner Join 0 to 1"}]
+ | keys:{"1":"_col0 (type: string)","0":"_col1 (type: string)"}
+ | outputColumnNames:["_col0"]
+ | Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
+ |<-Map 1 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_14]
+ | key expressions:_col1 (type: string)
+ | Map-reduce partition columns:_col1 (type: string)
+ | sort order:+
+ | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
+ | value expressions:_col0 (type: string)
+ | Map Join Operator [MAPJOIN_28]
+ | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | keys:{"Map 1":"_col0 (type: string)","Map 3":"_col0 (type: string)"}
+ | | outputColumnNames:["_col0","_col1"]
+ | | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
+ | |<-Map 3 [BROADCAST_EDGE]
+ | | Reduce Output Operator [RS_11]
+ | | key expressions:_col0 (type: string)
+ | | Map-reduce partition columns:_col0 (type: string)
+ | | sort order:+
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | Select Operator [SEL_4]
+ | | outputColumnNames:["_col0"]
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | Filter Operator [FIL_26]
+ | | predicate:key is not null (type: boolean)
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | TableScan [TS_3]
+ | | alias:src1
+ | | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+ | |<-Select Operator [SEL_2]
+ | outputColumnNames:["_col0","_col1"]
+ | Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
+ | Filter Operator [FIL_25]
+ | predicate:(((value > 'val_450') and key is not null) and value is not null) (type: boolean)
+ | Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
+ | TableScan [TS_0]
+ | alias:srcpart
+ | Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+ |<-Map 4 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_16]
+ key expressions:_col0 (type: string)
+ Map-reduce partition columns:_col0 (type: string)
+ sort order:+
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator [SEL_7]
+ outputColumnNames:["_col0"]
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator [FIL_27]
+ predicate:(value > 'val_450') (type: boolean)
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ TableScan [TS_5]
+ alias:src
+ Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
PREHOOK: query: explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'
PREHOOK: type: QUERY
@@ -6585,77 +6579,74 @@ POSTHOOK: type: QUERY
Plan optimized by CBO.
Vertex dependency in root stage
-Map 3 <- Map 4 (BROADCAST_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 3 (SIMPLE_EDGE)
+Map 1 <- Map 3 (BROADCAST_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
Reducer 2
- File Output Operator [FS_20]
+ File Output Operator [FS_19]
compressed:true
Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
- Select Operator [SEL_19]
- outputColumnNames:["_col0"]
- Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
- Merge Join Operator [MERGEJOIN_30]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col1 (type: string)","0":"_col0 (type: string)"}
- | outputColumnNames:["_col1"]
- | Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 1 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_15]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | Select Operator [SEL_2]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | Filter Operator [FIL_26]
- | predicate:(value > 'val_450') (type: boolean)
- | Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
- | TableScan [TS_0]
- | alias:src
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 3 [SIMPLE_EDGE]
- Reduce Output Operator [RS_17]
- key expressions:_col1 (type: string)
- Map-reduce partition columns:_col1 (type: string)
- sort order:+
- Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
- value expressions:_col0 (type: string)
- Map Join Operator [MAPJOIN_29]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"Map 3":"_col0 (type: string)","Map 4":"_col0 (type: string)"}
- | outputColumnNames:["_col0","_col1"]
- | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Map 4 [BROADCAST_EDGE]
- | Reduce Output Operator [RS_11]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | Select Operator [SEL_7]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | Filter Operator [FIL_28]
- | predicate:key is not null (type: boolean)
- | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
- | TableScan [TS_6]
- | alias:src1
- | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
- |<-Select Operator [SEL_5]
- outputColumnNames:["_col0","_col1"]
- Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
- Filter Operator [FIL_27]
- predicate:(((value > 'val_450') and key is not null) and value is not null) (type: boolean)
- Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
- TableScan [TS_3]
- alias:srcpart
- Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+ Merge Join Operator [MERGEJOIN_29]
+ | condition map:[{"":"Inner Join 0 to 1"}]
+ | keys:{"1":"_col0 (type: string)","0":"_col1 (type: string)"}
+ | outputColumnNames:["_col0"]
+ | Statistics:Num rows: 555 Data size: 48285 Basic stats: COMPLETE Column stats: COMPLETE
+ |<-Map 1 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_14]
+ | key expressions:_col1 (type: string)
+ | Map-reduce partition columns:_col1 (type: string)
+ | sort order:+
+ | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
+ | value expressions:_col0 (type: string)
+ | Map Join Operator [MAPJOIN_28]
+ | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | keys:{"Map 1":"_col0 (type: string)","Map 3":"_col0 (type: string)"}
+ | | outputColumnNames:["_col0","_col1"]
+ | | Statistics:Num rows: 241 Data size: 42898 Basic stats: COMPLETE Column stats: COMPLETE
+ | |<-Map 3 [BROADCAST_EDGE]
+ | | Reduce Output Operator [RS_11]
+ | | key expressions:_col0 (type: string)
+ | | Map-reduce partition columns:_col0 (type: string)
+ | | sort order:+
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | Select Operator [SEL_4]
+ | | outputColumnNames:["_col0"]
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | Filter Operator [FIL_26]
+ | | predicate:key is not null (type: boolean)
+ | | Statistics:Num rows: 25 Data size: 2150 Basic stats: COMPLETE Column stats: COMPLETE
+ | | TableScan [TS_3]
+ | | alias:src1
+ | | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: COMPLETE
+ | |<-Select Operator [SEL_2]
+ | outputColumnNames:["_col0","_col1"]
+ | Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
+ | Filter Operator [FIL_25]
+ | predicate:(((value > 'val_450') and key is not null) and value is not null) (type: boolean)
+ | Statistics:Num rows: 666 Data size: 118548 Basic stats: COMPLETE Column stats: COMPLETE
+ | TableScan [TS_0]
+ | alias:srcpart
+ | Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: COMPLETE
+ |<-Map 4 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_16]
+ key expressions:_col0 (type: string)
+ Map-reduce partition columns:_col0 (type: string)
+ sort order:+
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ Select Operator [SEL_7]
+ outputColumnNames:["_col0"]
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ Filter Operator [FIL_27]
+ predicate:(value > 'val_450') (type: boolean)
+ Statistics:Num rows: 166 Data size: 15106 Basic stats: COMPLETE Column stats: COMPLETE
+ TableScan [TS_5]
+ alias:src
+ Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: COMPLETE
PREHOOK: query: explain
select p_mfgr, p_name, p_size,
[11/25] hive git commit: HIVE-11055 HPL/SQL - Implementing Procedural
SQL in Hive (PL/HQL Contribution) (Dmitry Tolpeko via gates)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Select.java b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
new file mode 100644
index 0000000..e0f4098
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Select.java
@@ -0,0 +1,411 @@
+/**
+ * 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.hive.hplsql;
+
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.Stack;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.misc.Interval;
+
+public class Select {
+
+ Exec exec = null;
+ Stack<Var> stack = null;
+ Conf conf;
+
+ boolean trace = false;
+
+ Select(Exec e) {
+ exec = e;
+ stack = exec.getStack();
+ conf = exec.getConf();
+ trace = exec.getTrace();
+ }
+
+ /**
+ * Executing or building SELECT statement
+ */
+ public Integer select(HplsqlParser.Select_stmtContext ctx) {
+ if (ctx.parent instanceof HplsqlParser.StmtContext) {
+ exec.stmtConnList.clear();
+ trace(ctx, "SELECT");
+ }
+ boolean oldBuildSql = exec.buildSql;
+ exec.buildSql = true;
+ StringBuilder sql = new StringBuilder();
+ if (ctx.cte_select_stmt() != null) {
+ sql.append(evalPop(ctx.cte_select_stmt()).toString());
+ sql.append("\n");
+ }
+ sql.append(evalPop(ctx.fullselect_stmt()).toString());
+ exec.buildSql = oldBuildSql;
+ if (!(ctx.parent instanceof HplsqlParser.StmtContext)) { // No need to execute at this stage
+ exec.stackPush(sql);
+ return 0;
+ }
+ if (trace && ctx.parent instanceof HplsqlParser.StmtContext) {
+ trace(ctx, sql.toString());
+ }
+ if (exec.getOffline()) {
+ trace(ctx, "Not executed - offline mode set");
+ return 0;
+ }
+ String conn = exec.getStatementConnection();
+ Query query = exec.executeQuery(ctx, sql.toString(), conn);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ trace(ctx, "SELECT completed successfully");
+ exec.setSqlSuccess();
+ try {
+ ResultSet rs = query.getResultSet();
+ ResultSetMetaData rm = null;
+ if (rs != null) {
+ rm = rs.getMetaData();
+ }
+ HplsqlParser.Into_clauseContext into = getIntoClause(ctx);
+ if (into != null) {
+ trace(ctx, "SELECT INTO statement executed");
+ int cols = into.ident().size();
+ if (rs.next()) {
+ for (int i = 1; i <= cols; i++) {
+ Var var = exec.findVariable(into.ident(i-1).getText());
+ if (var != null) {
+ var.setValue(rs, rm, i);
+ if (trace) {
+ trace(ctx, "COLUMN: " + rm.getColumnName(i) + ", " + rm.getColumnTypeName(i));
+ trace(ctx, "SET " + var.getName() + " = " + var.toString());
+ }
+ }
+ else if(trace) {
+ trace(ctx, "Variable not found: " + into.ident(i-1).getText());
+ }
+ }
+ exec.incRowCount();
+ exec.setSqlSuccess();
+ }
+ else {
+ exec.setSqlCode(100);
+ exec.signal(Signal.Type.NOTFOUND);
+ }
+ }
+ // Print all results for standalone SELECT statement
+ else if (ctx.parent instanceof HplsqlParser.StmtContext) {
+ int cols = rm.getColumnCount();
+ if (trace) {
+ trace(ctx, "Standalone SELECT executed: " + cols + " columns in the result set");
+ }
+ while (rs.next()) {
+ for (int i = 1; i <= cols; i++) {
+ if (i > 1) {
+ System.out.print("\t");
+ }
+ System.out.print(rs.getString(i));
+ }
+ System.out.println("");
+ exec.incRowCount();
+ }
+ }
+ // Scalar subquery
+ else {
+ trace(ctx, "Scalar subquery executed, first row and first column fetched only");
+ if(rs.next()) {
+ exec.stackPush(new Var().setValue(rs, rm, 1));
+ exec.setSqlSuccess();
+ }
+ else {
+ evalNull();
+ exec.setSqlCode(100);
+ }
+ }
+ }
+ catch (SQLException e) {
+ exec.signal(query);
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 1;
+ }
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * Common table expression (WITH clause)
+ */
+ public Integer cte(HplsqlParser.Cte_select_stmtContext ctx) {
+ int cnt = ctx.cte_select_stmt_item().size();
+ StringBuilder sql = new StringBuilder();
+ sql.append("WITH ");
+ for (int i = 0; i < cnt; i++) {
+ HplsqlParser.Cte_select_stmt_itemContext c = ctx.cte_select_stmt_item(i);
+ sql.append(c.ident().getText());
+ if (c.cte_select_cols() != null) {
+ sql.append(" " + exec.getFormattedText(c.cte_select_cols()));
+ }
+ sql.append(" AS (");
+ sql.append(evalPop(ctx.cte_select_stmt_item(i).fullselect_stmt()).toString());
+ sql.append(")");
+ if (i + 1 != cnt) {
+ sql.append(",\n");
+ }
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * Part of SELECT
+ */
+ public Integer fullselect(HplsqlParser.Fullselect_stmtContext ctx) {
+ int cnt = ctx.fullselect_stmt_item().size();
+ StringBuilder sql = new StringBuilder();
+ for (int i = 0; i < cnt; i++) {
+ String part = evalPop(ctx.fullselect_stmt_item(i)).toString();
+ sql.append(part);
+ if (i + 1 != cnt) {
+ sql.append("\n" + getText(ctx.fullselect_set_clause(i)) + "\n");
+ }
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ public Integer subselect(HplsqlParser.Subselect_stmtContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ if (ctx.T_SELECT() != null) {
+ sql.append(ctx.T_SELECT().getText());
+ }
+ sql.append(" " + evalPop(ctx.select_list()));
+ if (ctx.from_clause() != null) {
+ sql.append(" " + evalPop(ctx.from_clause()));
+ } else {
+ sql.append(" FROM " + conf.dualTable);
+ }
+ if (ctx.where_clause() != null) {
+ sql.append(" " + evalPop(ctx.where_clause()));
+ }
+ if (ctx.group_by_clause() != null) {
+ sql.append(" " + getText(ctx.group_by_clause()));
+ }
+ if (ctx.having_clause() != null) {
+ sql.append(" " + getText(ctx.having_clause()));
+ }
+ if (ctx.order_by_clause() != null) {
+ sql.append(" " + getText(ctx.order_by_clause()));
+ }
+ if (ctx.select_options() != null) {
+ sql.append(" " + evalPop(ctx.select_options()));
+ }
+ if (ctx.select_list().select_list_limit() != null) {
+ sql.append(" LIMIT " + evalPop(ctx.select_list().select_list_limit().expr()));
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * SELECT list
+ */
+ public Integer selectList(HplsqlParser.Select_listContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ if (ctx.select_list_set() != null) {
+ sql.append(exec.getText(ctx.select_list_set())).append(" ");
+ }
+ int cnt = ctx.select_list_item().size();
+ for (int i = 0; i < cnt; i++) {
+ if (ctx.select_list_item(i).select_list_asterisk() == null) {
+ sql.append(evalPop(ctx.select_list_item(i)));
+ if (ctx.select_list_item(i).select_list_alias() != null) {
+ sql.append(" " + exec.getText(ctx.select_list_item(i).select_list_alias()));
+ }
+ }
+ else {
+ sql.append(exec.getText(ctx.select_list_item(i).select_list_asterisk()));
+ }
+ if (i + 1 < cnt) {
+ sql.append(", ");
+ }
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * FROM clause
+ */
+ public Integer from(HplsqlParser.From_clauseContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append(ctx.T_FROM().getText()).append(" ");
+ sql.append(evalPop(ctx.from_table_clause()));
+ int cnt = ctx.from_join_clause().size();
+ for (int i = 0; i < cnt; i++) {
+ sql.append(evalPop(ctx.from_join_clause(i)));
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * Single table name in FROM
+ */
+ public Integer fromTable(HplsqlParser.From_table_name_clauseContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append(evalPop(ctx.table_name()));
+ if (ctx.from_alias_clause() != null) {
+ sql.append(" ").append(exec.getText(ctx.from_alias_clause()));
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * JOIN clause in FROM
+ */
+ public Integer fromJoin(HplsqlParser.From_join_clauseContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ if (ctx.T_COMMA() != null) {
+ sql.append(", ");
+ sql.append(evalPop(ctx.from_table_clause()));
+ }
+ else if (ctx.from_join_type_clause() != null) {
+ sql.append(" ");
+ sql.append(exec.getText(ctx.from_join_type_clause()));
+ sql.append(" ");
+ sql.append(evalPop(ctx.from_table_clause()));
+ sql.append(" ");
+ sql.append(exec.getText(ctx, ctx.T_ON().getSymbol(), ctx.bool_expr().getStop()));
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * FROM TABLE (VALUES ...) clause
+ */
+ public Integer fromTableValues(HplsqlParser.From_table_values_clauseContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ int rows = ctx.from_table_values_row().size();
+ sql.append("(");
+ for (int i = 0; i < rows; i++) {
+ int cols = ctx.from_table_values_row(i).expr().size();
+ int cols_as = ctx.from_alias_clause().L_ID().size();
+ sql.append("SELECT ");
+ for (int j = 0; j < cols; j++) {
+ sql.append(evalPop(ctx.from_table_values_row(i).expr(j)));
+ if (j < cols_as) {
+ sql.append(" AS ");
+ sql.append(ctx.from_alias_clause().L_ID(j));
+ }
+ if (j + 1 < cols) {
+ sql.append(", ");
+ }
+ }
+ sql.append(" FROM " + conf.dualTable);
+ if (i + 1 < rows) {
+ sql.append("\nUNION ALL\n");
+ }
+ }
+ sql.append(") ");
+ if (ctx.from_alias_clause() != null) {
+ sql.append(ctx.from_alias_clause().ident().getText());
+ }
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * WHERE clause
+ */
+ public Integer where(HplsqlParser.Where_clauseContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append(ctx.T_WHERE().getText());
+ sql.append(" " + evalPop(ctx.bool_expr()));
+ exec.stackPush(sql);
+ return 0;
+ }
+
+ /**
+ * Get INTO clause
+ */
+ HplsqlParser.Into_clauseContext getIntoClause(HplsqlParser.Select_stmtContext ctx) {
+ if (ctx.fullselect_stmt().fullselect_stmt_item(0).subselect_stmt() != null) {
+ return ctx.fullselect_stmt().fullselect_stmt_item(0).subselect_stmt().into_clause();
+ }
+ return null;
+ }
+
+ /**
+ * SELECT statement options - LIMIT n, WITH UR i.e
+ */
+ public Integer option(HplsqlParser.Select_options_itemContext ctx) {
+ if (ctx.T_LIMIT() != null) {
+ exec.stackPush("LIMIT " + evalPop(ctx.expr()));
+ }
+ return 0;
+ }
+
+ /**
+ * Evaluate the expression to NULL
+ */
+ void evalNull() {
+ exec.stackPush(Var.Null);
+ }
+
+ /**
+ * Evaluate the expression and pop value from the stack
+ */
+ Var evalPop(ParserRuleContext ctx) {
+ exec.visit(ctx);
+ if (!exec.stack.isEmpty()) {
+ return exec.stackPop();
+ }
+ return Var.Empty;
+ }
+
+ /**
+ * Get node text including spaces
+ */
+ String getText(ParserRuleContext ctx) {
+ return ctx.start.getInputStream().getText(new Interval(ctx.start.getStartIndex(), ctx.stop.getStopIndex()));
+ }
+
+ /**
+ * Execute rules
+ */
+ Integer visit(ParserRuleContext ctx) {
+ return exec.visit(ctx);
+ }
+
+ /**
+ * Execute children rules
+ */
+ Integer visitChildren(ParserRuleContext ctx) {
+ return exec.visitChildren(ctx);
+ }
+
+ /**
+ * Trace information
+ */
+ void trace(ParserRuleContext ctx, String message) {
+ exec.trace(ctx, message);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java b/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
new file mode 100644
index 0000000..6330ae3
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Signal.java
@@ -0,0 +1,48 @@
+/**
+ * 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.hive.hplsql;
+
+/**
+ * Signals and exceptions
+ */
+public class Signal {
+ public enum Type { LEAVE_LOOP, LEAVE_ROUTINE, SQLEXCEPTION, NOTFOUND, USERDEFINED };
+ Type type;
+ String value = "";
+ Exception exception = null;
+
+ Signal(Type type, String value) {
+ this.type = type;
+ this.value = value;
+ this.exception = null;
+ }
+
+ Signal(Type type, String value, Exception exception) {
+ this.type = type;
+ this.value = value;
+ this.exception = exception;
+ }
+
+ /**
+ * Get the signal value (message text)
+ */
+ public String getValue() {
+ return value;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
new file mode 100644
index 0000000..acc4907
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Stmt.java
@@ -0,0 +1,1021 @@
+/**
+ * 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.hive.hplsql;
+
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.Stack;
+import java.util.UUID;
+
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hive.hplsql.Var.Type;
+import org.apache.hive.hplsql.HplsqlParser.Create_table_columns_itemContext;
+import org.apache.hive.hplsql.HplsqlParser.Create_table_columnsContext;
+
+/**
+ * HPL/SQL statements execution
+ */
+public class Stmt {
+
+ Exec exec = null;
+ Stack<Var> stack = null;
+ Conf conf;
+
+ boolean trace = false;
+
+ Stmt(Exec e) {
+ exec = e;
+ stack = exec.getStack();
+ conf = exec.getConf();
+ trace = exec.getTrace();
+ }
+
+ /**
+ * DECLARE cursor statement
+ */
+ public Integer declareCursor(HplsqlParser.Declare_cursor_itemContext ctx) {
+ String name = ctx.ident().getText();
+ if (trace) {
+ trace(ctx, "DECLARE CURSOR " + name);
+ }
+ Query query = new Query();
+ if (ctx.expr() != null) {
+ query.setExprCtx(ctx.expr());
+ }
+ else if (ctx.select_stmt() != null) {
+ query.setSelectCtx(ctx.select_stmt());
+ }
+ exec.addVariable(new Var(name, Type.CURSOR, query));
+ return 0;
+ }
+
+ /**
+ * CREATE TABLE statement
+ */
+ public Integer createTable(HplsqlParser.Create_table_stmtContext ctx) {
+ trace(ctx, "CREATE TABLE");
+ StringBuilder sql = new StringBuilder();
+ sql.append(exec.getText(ctx, ctx.T_CREATE().getSymbol(), ctx.T_OPEN_P().getSymbol()));
+ int cnt = ctx.create_table_columns().create_table_columns_item().size();
+ int cols = 0;
+ for (int i = 0; i < cnt; i++) {
+ Create_table_columns_itemContext col = ctx.create_table_columns().create_table_columns_item(i);
+ if (col.create_table_column_cons() != null) {
+ continue;
+ }
+ if (cols > 0) {
+ sql.append(",\n");
+ }
+ sql.append(col.ident().getText());
+ sql.append(" ");
+ sql.append(exec.evalPop(col.dtype(), col.dtype_len()));
+ cols++;
+ }
+ sql.append("\n)");
+ if (ctx.create_table_options() != null) {
+ sql.append(" " + evalPop(ctx.create_table_options()).toString());
+ }
+ trace(ctx, sql.toString());
+ Query query = exec.executeSql(ctx, sql.toString(), exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * CREATE TABLE options for Hive
+ */
+ public Integer createTableHiveOptions(HplsqlParser.Create_table_options_hive_itemContext ctx) {
+ if (ctx.create_table_hive_row_format() != null) {
+ createTableHiveRowFormat(ctx.create_table_hive_row_format());
+ }
+ return 0;
+ }
+
+ public Integer createTableHiveRowFormat(HplsqlParser.Create_table_hive_row_formatContext ctx) {
+ StringBuilder sql = new StringBuilder();
+ sql.append("ROW FORMAT DELIMITED");
+ int cnt = ctx.create_table_hive_row_format_fields().size();
+ for (int i = 0; i < cnt; i++) {
+ HplsqlParser.Create_table_hive_row_format_fieldsContext c = ctx.create_table_hive_row_format_fields(i);
+ if (c.T_FIELDS() != null) {
+ sql.append(" FIELDS TERMINATED BY " + evalPop(c.expr(0)).toSqlString());
+ }
+ else if (c.T_LINES() != null) {
+ sql.append(" LINES TERMINATED BY " + evalPop(c.expr(0)).toSqlString());
+ }
+ }
+ evalString(sql);
+ return 0;
+ }
+
+ /**
+ * DECLARE TEMPORARY TABLE statement
+ */
+ public Integer declareTemporaryTable(HplsqlParser.Declare_temporary_table_itemContext ctx) {
+ String name = ctx.ident().getText();
+ if (trace) {
+ trace(ctx, "DECLARE TEMPORARY TABLE " + name);
+ }
+ return createTemporaryTable(ctx, ctx.create_table_columns(), name);
+ }
+
+ /**
+ * CREATE LOCAL TEMPORARY | VOLATILE TABLE statement
+ */
+ public Integer createLocalTemporaryTable(HplsqlParser.Create_local_temp_table_stmtContext ctx) {
+ String name = ctx.ident().getText();
+ if (trace) {
+ trace(ctx, "CREATE LOCAL TEMPORARY TABLE " + name);
+ }
+ return createTemporaryTable(ctx, ctx.create_table_columns(), name);
+ }
+
+ /**
+ * Create a temporary table statement
+ */
+ public Integer createTemporaryTable(ParserRuleContext ctx, Create_table_columnsContext colCtx, String name) {
+ String managedName = null;
+ String sql = null;
+ String columns = exec.getFormattedText(colCtx);
+ if (conf.tempTables == Conf.TempTables.NATIVE) {
+ sql = "CREATE TEMPORARY TABLE " + name + "\n(" + columns + "\n)";
+ } else if (conf.tempTables == Conf.TempTables.MANAGED) {
+ managedName = name + "_" + UUID.randomUUID().toString().replace("-","");
+ if (!conf.tempTablesSchema.isEmpty()) {
+ managedName = conf.tempTablesSchema + "." + managedName;
+ }
+ sql = "CREATE TABLE " + managedName + "\n(" + columns + "\n)";
+ if (!conf.tempTablesLocation.isEmpty()) {
+ sql += "\nLOCATION '" + conf.tempTablesLocation + "/" + managedName + "'";
+ }
+ if (trace) {
+ trace(ctx, "Managed table name: " + managedName);
+ }
+ }
+ if (sql != null) {
+ Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ if (managedName != null) {
+ exec.addManagedTable(name, managedName);
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ }
+ return 0;
+ }
+
+ /**
+ * DROP statement
+ */
+ public Integer drop(HplsqlParser.Drop_stmtContext ctx) {
+ trace(ctx, "DROP");
+ String sql = null;
+ if (ctx.T_TABLE() != null) {
+ sql = "DROP TABLE ";
+ if (ctx.T_EXISTS() != null) {
+ sql += "IF NOT EXISTS ";
+ }
+ sql += evalPop(ctx.table_name()).toString();
+ }
+ if (sql != null) {
+ trace(ctx, sql);
+ Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ }
+ return 0;
+ }
+
+ /**
+ * OPEN cursor statement
+ */
+ public Integer open(HplsqlParser.Open_stmtContext ctx) {
+ trace(ctx, "OPEN");
+ Query query = null;
+ Var var = null;
+ String cursor = ctx.L_ID().toString();
+ String sql = null;
+ // Dynamic SQL
+ if (ctx.T_FOR() != null) {
+ sql = evalPop(ctx.expr()).toString();
+ if (trace) {
+ trace(ctx, cursor + ": " + sql);
+ }
+ query = new Query(sql);
+ var = new Var(cursor, Type.CURSOR, query);
+ exec.addVariable(var);
+ }
+ // Declared cursor
+ else {
+ var = exec.findVariable(cursor);
+ if (var != null && var.type == Type.CURSOR) {
+ query = (Query)var.value;
+ if (query.sqlExpr != null) {
+ sql = evalPop(query.sqlExpr).toString();
+ query.setSql(sql);
+ }
+ else if (query.sqlSelect != null) {
+ sql = evalPop(query.sqlSelect).toString();
+ query.setSql(sql);
+ }
+ if (trace) {
+ trace(ctx, cursor + ": " + sql);
+ }
+ }
+ }
+ // Open cursor now
+ if (query != null) {
+ exec.executeQuery(ctx, query, exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ else if (!exec.getOffline()) {
+ exec.setSqlCode(0);
+ }
+ }
+ else {
+ trace(ctx, "Cursor not found: " + cursor);
+ exec.setSqlCode(-1);
+ exec.signal(Signal.Type.SQLEXCEPTION);
+ return 1;
+ }
+ return 0;
+ }
+
+ /**
+ * FETCH cursor statement
+ */
+ public Integer fetch(HplsqlParser.Fetch_stmtContext ctx) {
+ trace(ctx, "FETCH");
+ String name = ctx.L_ID(0).toString();
+ Var cursor = exec.findVariable(name);
+ if (cursor == null || cursor.type != Type.CURSOR) {
+ trace(ctx, "Cursor not found: " + name);
+ exec.setSqlCode(-1);
+ exec.signal(Signal.Type.SQLEXCEPTION);
+ return 1;
+ }
+ else if (exec.getOffline()) {
+ exec.setSqlCode(100);
+ exec.signal(Signal.Type.NOTFOUND);
+ return 0;
+ }
+ // Assign values from the row to local variables
+ try {
+ Query query = (Query)cursor.value;
+ ResultSet rs = query.getResultSet();
+ ResultSetMetaData rsm = null;
+ if(rs != null) {
+ rsm = rs.getMetaData();
+ }
+ if(rs != null && rsm != null) {
+ int cols = ctx.L_ID().size() - 1;
+ if(rs.next()) {
+ for(int i=1; i <= cols; i++) {
+ Var var = exec.findVariable(ctx.L_ID(i).getText());
+ if(var != null) {
+ var.setValue(rs, rsm, i);
+ if(trace) {
+ trace(ctx, "COLUMN: " + rsm.getColumnName(i) + ", " + rsm.getColumnTypeName(i));
+ trace(ctx, "SET " + var.getName() + " = " + var.toString());
+ }
+ }
+ else if(trace) {
+ trace(ctx, "Variable not found: " + ctx.L_ID(i).getText());
+ }
+ }
+ exec.incRowCount();
+ exec.setSqlSuccess();
+ }
+ else {
+ exec.setSqlCode(100);
+ exec.signal(Signal.Type.NOTFOUND);
+ }
+ }
+ }
+ catch (SQLException e) {
+ exec.setSqlCode(e);
+ exec.signal(Signal.Type.SQLEXCEPTION, e.getMessage(), e);
+ }
+ return 0;
+ }
+
+ /**
+ * CLOSE cursor statement
+ */
+ public Integer close(HplsqlParser.Close_stmtContext ctx) {
+ trace(ctx, "CLOSE");
+ String name = ctx.L_ID().toString();
+ Var var = exec.findVariable(name);
+ if(var != null && var.type == Type.CURSOR) {
+ exec.closeQuery((Query)var.value, exec.conf.defaultConnection);
+ exec.setSqlCode(0);
+ }
+ else if(trace) {
+ trace(ctx, "Cursor not found: " + name);
+ }
+ return 0;
+ }
+
+ /**
+ * INCLUDE statement
+ */
+ public Integer include(HplsqlParser.Include_stmtContext ctx) {
+ String file = ctx.file_name().getText();
+ trace(ctx, "INCLUDE " + file);
+ exec.includeFile(file);
+ return 0;
+ }
+
+ /**
+ * IF statement (PL/SQL syntax)
+ */
+ public Integer ifPlsql(HplsqlParser.If_plsql_stmtContext ctx) {
+ boolean trueExecuted = false;
+ trace(ctx, "IF");
+ if (evalPop(ctx.bool_expr()).isTrue()) {
+ trace(ctx, "IF TRUE executed");
+ visit(ctx.block());
+ trueExecuted = true;
+ }
+ else if (ctx.elseif_block() != null) {
+ int cnt = ctx.elseif_block().size();
+ for (int i = 0; i < cnt; i++) {
+ if (evalPop(ctx.elseif_block(i).bool_expr()).isTrue()) {
+ trace(ctx, "ELSE IF executed");
+ visit(ctx.elseif_block(i).block());
+ trueExecuted = true;
+ break;
+ }
+ }
+ }
+ if (!trueExecuted && ctx.else_block() != null) {
+ trace(ctx, "ELSE executed");
+ visit(ctx.else_block());
+ }
+ return 0;
+ }
+
+ /**
+ * IF statement (Transact-SQL syntax)
+ */
+ public Integer ifTsql(HplsqlParser.If_tsql_stmtContext ctx) {
+ trace(ctx, "IF");
+ visit(ctx.bool_expr());
+ if(exec.stackPop().isTrue()) {
+ trace(ctx, "IF TRUE executed");
+ visit(ctx.single_block_stmt(0));
+ }
+ else if(ctx.T_ELSE() != null) {
+ trace(ctx, "ELSE executed");
+ visit(ctx.single_block_stmt(1));
+ }
+ return 0;
+ }
+
+ /**
+ * Assignment from SELECT statement
+ */
+ public Integer assignFromSelect(HplsqlParser.Assignment_stmt_select_itemContext ctx) {
+ String sql = evalPop(ctx.select_stmt()).toString();
+ if (trace) {
+ trace(ctx, sql.toString());
+ }
+ String conn = exec.getStatementConnection();
+ Query query = exec.executeQuery(ctx, sql.toString(), conn);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ try {
+ ResultSet rs = query.getResultSet();
+ ResultSetMetaData rm = null;
+ if (rs != null) {
+ rm = rs.getMetaData();
+ int cnt = ctx.ident().size();
+ if (rs.next()) {
+ for (int i = 1; i <= cnt; i++) {
+ Var var = exec.findVariable(ctx.ident(i-1).getText());
+ if (var != null) {
+ var.setValue(rs, rm, i);
+ if (trace) {
+ trace(ctx, "COLUMN: " + rm.getColumnName(i) + ", " + rm.getColumnTypeName(i));
+ trace(ctx, "SET " + var.getName() + " = " + var.toString());
+ }
+ }
+ else if(trace) {
+ trace(ctx, "Variable not found: " + ctx.ident(i-1).getText());
+ }
+ }
+ exec.incRowCount();
+ exec.setSqlSuccess();
+ }
+ else {
+ exec.setSqlCode(100);
+ exec.signal(Signal.Type.NOTFOUND);
+ }
+ }
+ }
+ catch (SQLException e) {
+ exec.signal(query);
+ return 1;
+ }
+ finally {
+ exec.closeQuery(query, conn);
+ }
+ return 0;
+ }
+
+ /**
+ * SQL INSERT statement
+ */
+ public Integer insert(HplsqlParser.Insert_stmtContext ctx) {
+ exec.stmtConnList.clear();
+ if (ctx.select_stmt() != null) {
+ return insertSelect(ctx);
+ }
+ return insertValues(ctx);
+ }
+
+ /**
+ * SQL INSERT SELECT statement
+ */
+ public Integer insertSelect(HplsqlParser.Insert_stmtContext ctx) {
+ trace(ctx, "INSERT SELECT");
+ String table = evalPop(ctx.table_name()).toString();
+ String select = evalPop(ctx.select_stmt()).toString();
+ String sql = "INSERT INTO TABLE " + table + " " + select;
+ trace(ctx, sql);
+ Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * SQL INSERT VALUES statement
+ */
+ public Integer insertValues(HplsqlParser.Insert_stmtContext ctx) {
+ trace(ctx, "INSERT VALUES");
+ String table = evalPop(ctx.table_name()).toString();
+ String conn = exec.getObjectConnection(ctx.table_name().getText());
+ Conn.Type type = exec.getConnectionType(conn);
+ StringBuilder sql = new StringBuilder();
+ if (type == Conn.Type.HIVE) {
+ sql.append("INSERT INTO TABLE " + table + " ");
+ if (conf.insertValues == Conf.InsertValues.NATIVE) {
+ sql.append("VALUES\n(");
+ }
+ }
+ else {
+ sql.append("INSERT INTO " + table);
+ if (ctx.insert_stmt_cols() != null) {
+ sql.append(" " + exec.getFormattedText(ctx.insert_stmt_cols()));
+ }
+ sql.append(" VALUES\n(");
+ }
+ int rows = ctx.insert_stmt_rows().insert_stmt_row().size();
+ for (int i = 0; i < rows; i++) {
+ HplsqlParser.Insert_stmt_rowContext row =ctx.insert_stmt_rows().insert_stmt_row(i);
+ int cols = row.expr().size();
+ for (int j = 0; j < cols; j++) {
+ String value = evalPop(row.expr(j)).toSqlString();
+ if (j == 0 && type == Conn.Type.HIVE && conf.insertValues == Conf.InsertValues.SELECT ) {
+ sql.append("SELECT ");
+ }
+ sql.append(value);
+ if (j + 1 != cols) {
+ sql.append(", ");
+ }
+ }
+ if (type != Conn.Type.HIVE || conf.insertValues == Conf.InsertValues.NATIVE) {
+ if (i + 1 == rows) {
+ sql.append(")");
+ } else {
+ sql.append("),\n(");
+ }
+ }
+ else if (type == Conn.Type.HIVE && conf.insertValues == Conf.InsertValues.SELECT) {
+ sql.append(" FROM " + conf.dualTable);
+ if (i + 1 < rows) {
+ sql.append("\nUNION ALL\n");
+ }
+ }
+ }
+ if (trace) {
+ trace(ctx, sql.toString());
+ }
+ Query query = exec.executeSql(ctx, sql.toString(), conn);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * GET DIAGNOSTICS EXCEPTION statement
+ */
+ public Integer getDiagnosticsException(HplsqlParser.Get_diag_stmt_exception_itemContext ctx) {
+ trace(ctx, "GET DIAGNOSTICS EXCEPTION");
+ Signal signal = exec.signalPeek();
+ if (signal == null || (signal != null && signal.type != Signal.Type.SQLEXCEPTION)) {
+ signal = exec.currentSignal;
+ }
+ if (signal != null) {
+ exec.setVariable(ctx.ident().getText(), signal.getValue());
+ }
+ return 0;
+ }
+
+ /**
+ * GET DIAGNOSTICS ROW_COUNT statement
+ */
+ public Integer getDiagnosticsRowCount(HplsqlParser.Get_diag_stmt_rowcount_itemContext ctx) {
+ trace(ctx, "GET DIAGNOSTICS ROW_COUNT");
+ exec.setVariable(ctx.ident().getText(), exec.getRowCount());
+ return 0;
+ }
+
+ /**
+ * USE statement
+ */
+ public Integer use(HplsqlParser.Use_stmtContext ctx) {
+ if(trace) {
+ trace(ctx, "USE");
+ }
+ String sql = ctx.T_USE().toString() + " " + evalPop(ctx.expr()).toString();
+ if(trace) {
+ trace(ctx, "Query: " + sql);
+ }
+ Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+ if(query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlCode(0);
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * VALUES statement
+ */
+ public Integer values(HplsqlParser.Values_into_stmtContext ctx) {
+ trace(ctx, "VALUES statement");
+ int cnt = ctx.ident().size(); // Number of variables and assignment expressions
+ int ecnt = ctx.expr().size();
+ for (int i = 0; i < cnt; i++) {
+ String name = ctx.ident(i).getText();
+ if (i < ecnt) {
+ visit(ctx.expr(i));
+ Var var = exec.setVariable(name);
+ if (trace) {
+ trace(ctx, "SET " + name + " = " + var.toString());
+ }
+ }
+ }
+ return 0;
+ }
+
+ /**
+ * WHILE statement
+ */
+ public Integer while_(HplsqlParser.While_stmtContext ctx) {
+ trace(ctx, "WHILE - ENTERED");
+ String label = exec.labelPop();
+ while (true) {
+ if (evalPop(ctx.bool_expr()).isTrue()) {
+ exec.enterScope(Scope.Type.LOOP);
+ visit(ctx.block());
+ exec.leaveScope();
+ if (canContinue(label)) {
+ continue;
+ }
+ }
+ break;
+ }
+ trace(ctx, "WHILE - LEFT");
+ return 0;
+ }
+
+ /**
+ * FOR cursor statement
+ */
+ public Integer forCursor(HplsqlParser.For_cursor_stmtContext ctx) {
+ trace(ctx, "FOR CURSOR - ENTERED");
+ exec.enterScope(Scope.Type.LOOP);
+ String cursor = ctx.L_ID().getText();
+ String sql = evalPop(ctx.select_stmt()).toString();
+ trace(ctx, sql);
+ Query query = exec.executeQuery(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ trace(ctx, "SELECT completed successfully");
+ exec.setSqlSuccess();
+ try {
+ ResultSet rs = query.getResultSet();
+ if (rs != null) {
+ ResultSetMetaData rm = rs.getMetaData();
+ int cols = rm.getColumnCount();
+ Var[] vars = new Var[cols];
+ for (int i = 0; i < cols; i++) {
+ vars[i] = new Var();
+ vars[i].setName(cursor + "." + rm.getColumnName(i + 1));
+ vars[i].setType(rm.getColumnType(i + 1));
+ exec.addVariable(vars[i]);
+ if (trace) {
+ trace(ctx, "Column: " + vars[i].getName() + " " + rm.getColumnTypeName(i + 1));
+ }
+ }
+ while (rs.next()) {
+ for (int i = 0; i < cols; i++) {
+ vars[i].setValue(rs, rm, i + 1);
+ }
+ visit(ctx.block());
+ exec.incRowCount();
+ }
+ }
+ }
+ catch (SQLException e) {
+ exec.signal(e);
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ exec.leaveScope();
+ trace(ctx, "FOR CURSOR - LEFT");
+ return 0;
+ }
+
+ /**
+ * FOR (integer range) statement
+ */
+ public Integer forRange(HplsqlParser.For_range_stmtContext ctx) {
+ trace(ctx, "FOR RANGE - ENTERED");
+ int start = evalPop(ctx.expr(0)).intValue();
+ int end = evalPop(ctx.expr(1)).intValue();
+ int step = evalPop(ctx.expr(2), 1L).intValue();
+ exec.enterScope(Scope.Type.LOOP);
+ Var index = new Var(ctx.L_ID().getText(), new Long(start));
+ exec.addVariable(index);
+ if (ctx.T_REVERSE() == null) {
+ for (int i = start; i <= end; i += step) {
+ visit(ctx.block());
+ index.increment(new Long(step));
+ }
+ } else {
+ for (int i = start; i >= end; i -= step) {
+ visit(ctx.block());
+ index.decrement(new Long(step));
+ }
+ }
+ exec.leaveScope();
+ trace(ctx, "FOR RANGE - LEFT");
+ return 0;
+ }
+
+ /**
+ * EXEC, EXECUTE and EXECUTE IMMEDIATE statement to execute dynamic SQL
+ */
+ public Integer exec(HplsqlParser.Exec_stmtContext ctx) {
+ if(trace) {
+ trace(ctx, "EXECUTE");
+ }
+ Var vsql = evalPop(ctx.expr());
+ String sql = vsql.toString();
+ if(trace) {
+ trace(ctx, "Query: " + sql);
+ }
+ Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+ if(query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ ResultSet rs = query.getResultSet();
+ if(rs != null) {
+ try {
+ ResultSetMetaData rsm = rs.getMetaData();
+ // Assign to variables
+ if(ctx.T_INTO() != null) {
+ int cols = ctx.L_ID().size();
+ if(rs.next()) {
+ for(int i=0; i < cols; i++) {
+ Var var = exec.findVariable(ctx.L_ID(i).getText());
+ if(var != null) {
+ var.setValue(rs, rsm, i+1);
+ if(trace) {
+ trace(ctx, "COLUMN: " + rsm.getColumnName(i+1) + ", " + rsm.getColumnTypeName(i+1));
+ trace(ctx, "SET " + var.getName() + " = " + var.toString());
+ }
+ }
+ else if(trace) {
+ trace(ctx, "Variable not found: " + ctx.L_ID(i).getText());
+ }
+ }
+ exec.setSqlCode(0);
+ }
+ }
+ // Print the results
+ else {
+ int cols = rsm.getColumnCount();
+ while(rs.next()) {
+ for(int i = 1; i <= cols; i++) {
+ if(i > 1) {
+ System.out.print("\t");
+ }
+ System.out.print(rs.getString(i));
+ }
+ System.out.println("");
+ }
+ }
+ }
+ catch(SQLException e) {
+ exec.setSqlCode(e);
+ }
+ }
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * EXIT statement (leave the specified loop with a condition)
+ */
+ public Integer exit(HplsqlParser.Exit_stmtContext ctx) {
+ trace(ctx, "EXIT");
+ String label = "";
+ if (ctx.L_ID() != null) {
+ label = ctx.L_ID().toString();
+ }
+ if (ctx.T_WHEN() != null) {
+ if (evalPop(ctx.bool_expr()).isTrue()) {
+ leaveLoop(label);
+ }
+ } else {
+ leaveLoop(label);
+ }
+ return 0;
+ }
+
+ /**
+ * BREAK statement (leave the innermost loop unconditionally)
+ */
+ public Integer break_(HplsqlParser.Break_stmtContext ctx) {
+ trace(ctx, "BREAK");
+ leaveLoop("");
+ return 0;
+ }
+
+ /**
+ * LEAVE statement (leave the specified loop unconditionally)
+ */
+ public Integer leave(HplsqlParser.Leave_stmtContext ctx) {
+ trace(ctx, "LEAVE");
+ String label = "";
+ if (ctx.L_ID() != null) {
+ label = ctx.L_ID().toString();
+ }
+ leaveLoop(label);
+ return 0;
+ }
+
+ /**
+ * Leave the specified or innermost loop unconditionally
+ */
+ public void leaveLoop(String value) {
+ exec.signal(Signal.Type.LEAVE_LOOP, value);
+ }
+
+ /**
+ * UPDATE statement
+ */
+ public Integer update(HplsqlParser.Update_stmtContext ctx) {
+ trace(ctx, "UPDATE");
+ String sql = exec.getFormattedText(ctx);
+ trace(ctx, sql);
+ Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * DELETE statement
+ */
+ public Integer delete(HplsqlParser.Delete_stmtContext ctx) {
+ trace(ctx, "DELETE");
+ String table = evalPop(ctx.table_name()).toString();
+ StringBuilder sql = new StringBuilder();
+ sql.append("DELETE FROM ");
+ sql.append(table);
+ if (ctx.where_clause() != null) {
+ boolean oldBuildSql = exec.buildSql;
+ exec.buildSql = true;
+ sql.append(" " + evalPop(ctx.where_clause()).toString());
+ exec.buildSql = oldBuildSql;
+ }
+ trace(ctx, sql.toString());
+ Query query = exec.executeSql(ctx, sql.toString(), exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * MERGE statement
+ */
+ public Integer merge(HplsqlParser.Merge_stmtContext ctx) {
+ trace(ctx, "MERGE");
+ String sql = exec.getFormattedText(ctx);
+ trace(ctx, sql);
+ Query query = exec.executeSql(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ exec.signal(query);
+ return 1;
+ }
+ exec.setSqlSuccess();
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ return 0;
+ }
+
+ /**
+ * PRINT Statement
+ */
+ public Integer print(HplsqlParser.Print_stmtContext ctx) {
+ trace(ctx, "PRINT");
+ if (ctx.expr() != null) {
+ visit(ctx.expr());
+ System.out.println(stack.pop().toString());
+ }
+ return 0;
+ }
+
+ /**
+ * SIGNAL statement
+ */
+ public Integer signal(HplsqlParser.Signal_stmtContext ctx) {
+ trace(ctx, "SIGNAL");
+ Signal signal = new Signal(Signal.Type.USERDEFINED, ctx.ident().getText());
+ exec.signal(signal);
+ return 0;
+ }
+
+ /**
+ * RESIGNAL statement
+ */
+ public Integer resignal(HplsqlParser.Resignal_stmtContext ctx) {
+ trace(ctx, "RESIGNAL");
+ if (ctx.T_SQLSTATE() != null) {
+ String sqlstate = evalPop(ctx.expr(0)).toString();
+ String text = "";
+ if (ctx.T_MESSAGE_TEXT() != null) {
+ text = evalPop(ctx.expr(1)).toString();
+ }
+ SQLException exception = new SQLException(text, sqlstate, -1);
+ Signal signal = new Signal(Signal.Type.SQLEXCEPTION, text, exception);
+ exec.setSqlCode(exception);
+ exec.resignal(signal);
+ }
+ else {
+ exec.resignal();
+ }
+ return 0;
+ }
+
+ /**
+ * RETURN statement
+ */
+ public Integer return_(HplsqlParser.Return_stmtContext ctx) {
+ trace(ctx, "RETURN");
+ if (ctx.expr() != null) {
+ eval(ctx.expr());
+ }
+ exec.signal(Signal.Type.LEAVE_ROUTINE);
+ return 0;
+ }
+
+ /**
+ * Check if an exception is raised or EXIT executed, and we should leave the block
+ */
+ boolean canContinue(String label) {
+ Signal signal = exec.signalPeek();
+ if (signal != null && signal.type == Signal.Type.SQLEXCEPTION) {
+ return false;
+ }
+ signal = exec.signalPeek();
+ if (signal != null && signal.type == Signal.Type.LEAVE_LOOP) {
+ if (signal.value == null || signal.value.isEmpty() ||
+ (label != null && label.equalsIgnoreCase(signal.value))) {
+ exec.signalPop();
+ }
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * Evaluate the expression and push the value to the stack
+ */
+ void eval(ParserRuleContext ctx) {
+ exec.visit(ctx);
+ }
+
+ /**
+ * Evaluate the expression to specified String value
+ */
+ void evalString(String string) {
+ exec.stackPush(new Var(string));
+ }
+
+ void evalString(StringBuilder string) {
+ evalString(string.toString());
+ }
+
+ /**
+ * Evaluate the expression and pop value from the stack
+ */
+ Var evalPop(ParserRuleContext ctx) {
+ visit(ctx);
+ if (!exec.stack.isEmpty()) {
+ return exec.stackPop();
+ }
+ return Var.Empty;
+ }
+
+ Var evalPop(ParserRuleContext ctx, long def) {
+ if (ctx != null) {
+ exec.visit(ctx);
+ return exec.stackPop();
+ }
+ return new Var(def);
+ }
+
+ /**
+ * Execute rules
+ */
+ Integer visit(ParserRuleContext ctx) {
+ return exec.visit(ctx);
+ }
+
+ /**
+ * Execute children rules
+ */
+ Integer visitChildren(ParserRuleContext ctx) {
+ return exec.visitChildren(ctx);
+ }
+
+ /**
+ * Trace information
+ */
+ void trace(ParserRuleContext ctx, String message) {
+ exec.trace(ctx, message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.java b/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.java
new file mode 100644
index 0000000..d5a7cc4
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/StreamGobbler.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.hive.hplsql;
+
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.BufferedReader;
+import java.io.IOException;
+
+/**
+ * Read a stream from an external process
+ */
+public class StreamGobbler extends Thread {
+ InputStream is;
+
+ StreamGobbler(InputStream is) {
+ this.is = is;
+ }
+
+ public void run() {
+ try {
+ InputStreamReader isr = new InputStreamReader(is);
+ BufferedReader br = new BufferedReader(isr);
+ while(true) {
+ String line = br.readLine();
+ if(line == null) {
+ break;
+ }
+ System.out.println(line);
+ }
+ } catch (IOException ioe) {
+ ioe.printStackTrace();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java b/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java
new file mode 100644
index 0000000..9330eb4
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Timer.java
@@ -0,0 +1,59 @@
+/**
+ * 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.hive.hplsql;
+
+public class Timer {
+ long start = 0;
+ long stop = 0;
+ long elapsed = 0;
+
+ /**
+ * Start the timer
+ */
+ public long start() {
+ start = System.currentTimeMillis();
+ return start;
+ }
+
+ /**
+ * Get intermediate timer value
+ */
+ public long current() {
+ return System.currentTimeMillis();
+ }
+
+ /**
+ * Stop the timer and return elapsed time
+ */
+ public long stop() {
+ stop = System.currentTimeMillis();
+ elapsed = stop - start;
+ return elapsed;
+ }
+
+ /**
+ * Format the elapsed time
+ */
+ public String format() {
+ if (elapsed < 1000) {
+ return String.valueOf(elapsed) + " ms";
+ }
+ return String.format("%.2f", ((float)elapsed)/1000) + " sec";
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java b/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java
new file mode 100644
index 0000000..9c29eeb
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Udf.java
@@ -0,0 +1,117 @@
+/**
+ * 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.hive.hplsql;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentLengthException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDF;
+import org.apache.hadoop.hive.ql.udf.UDFType;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+
+@Description(name = "hplsql", value = "_FUNC_('query' [, :1, :2, ...n]) - Execute HPL/SQL query", extended = "Example:\n" + " > SELECT _FUNC_('CURRENT_DATE') FROM src LIMIT 1;\n")
+@UDFType(deterministic = false)
+public class Udf extends GenericUDF {
+
+ Exec exec;
+ StringObjectInspector queryOI;
+ ObjectInspector[] argumentsOI;
+
+ /**
+ * Initialize UDF
+ */
+ @Override
+ public ObjectInspector initialize(ObjectInspector[] arguments) throws UDFArgumentException {
+ if (arguments.length == 0) {
+ throw new UDFArgumentLengthException("At least one argument must be specified");
+ }
+ if (!(arguments[0] instanceof StringObjectInspector)) {
+ throw new UDFArgumentException("First argument must be a string");
+ }
+ queryOI = (StringObjectInspector)arguments[0];
+ argumentsOI = arguments;
+ return PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+ }
+
+ /**
+ * Execute UDF
+ */
+ @Override
+ public Object evaluate(DeferredObject[] arguments) throws HiveException {
+ if (exec == null) {
+ exec = new Exec();
+ String query = queryOI.getPrimitiveJavaObject(arguments[0].get());
+ String[] args = { "-e", query, "-trace" };
+ try {
+ exec.setUdfRun(true);
+ exec.init(args);
+ } catch (Exception e) {
+ throw new HiveException(e.getMessage());
+ }
+ }
+ if (arguments.length > 1) {
+ setParameters(arguments);
+ }
+ Var result = exec.run();
+ if (result != null) {
+ return result.toString();
+ }
+ return null;
+ }
+
+ /**
+ * Set parameters for the current call
+ */
+ void setParameters(DeferredObject[] arguments) throws HiveException {
+ for (int i = 1; i < arguments.length; i++) {
+ String name = ":" + i;
+ if (argumentsOI[i] instanceof StringObjectInspector) {
+ String value = ((StringObjectInspector)argumentsOI[i]).getPrimitiveJavaObject(arguments[i].get());
+ if (value != null) {
+ exec.setVariable(name, value);
+ }
+ }
+ else if (argumentsOI[i] instanceof IntObjectInspector) {
+ Integer value = (Integer)((IntObjectInspector)argumentsOI[i]).getPrimitiveJavaObject(arguments[i].get());
+ if (value != null) {
+ exec.setVariable(name, new Var(new Long(value)));
+ }
+ }
+ else if (argumentsOI[i] instanceof LongObjectInspector) {
+ Long value = (Long)((LongObjectInspector)argumentsOI[i]).getPrimitiveJavaObject(arguments[i].get());
+ if (value != null) {
+ exec.setVariable(name, new Var(value));
+ }
+ }
+ else {
+ exec.setVariableToNull(name);
+ }
+ }
+ }
+
+ @Override
+ public String getDisplayString(String[] children) {
+ return "hplsql";
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java b/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
new file mode 100644
index 0000000..da0d878
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Utils.java
@@ -0,0 +1,289 @@
+/**
+ * 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.hive.hplsql;
+
+import java.sql.Date;
+import java.sql.Timestamp;
+
+public class Utils {
+
+ /**
+ * Unquote string and remove escape characters inside the script
+ */
+ public static String unquoteString(String s) {
+ if(s == null) {
+ return null;
+ }
+
+ int len = s.length();
+ StringBuffer s2 = new StringBuffer(len);
+
+ for (int i = 0; i < len; i++) {
+ char ch = s.charAt(i);
+ char ch2 = (i < len - 1) ? s.charAt(i+1) : 0;
+
+ if((i == 0 || i == len -1) && (ch == '\'' || ch == '"'))
+ continue;
+ else
+ // \' and '' escape sequences
+ if((ch == '\\' && ch2 == '\'') || (ch == '\'' && ch2 == '\''))
+ continue;
+
+ s2.append(ch);
+ }
+
+ return s2.toString();
+ }
+
+ /**
+ * Quote string and escape characters - ab'c -> 'ab''c'
+ */
+ public static String quoteString(String s) {
+ if(s == null) {
+ return null;
+ }
+ int len = s.length();
+ StringBuilder s2 = new StringBuilder(len + 2).append('\'');
+
+ for (int i = 0; i < len; i++) {
+ char ch = s.charAt(i);
+ s2.append(ch);
+ if(ch == '\'') {
+ s2.append(ch);
+ }
+ }
+ s2.append('\'');
+ return s2.toString();
+ }
+
+ /**
+ * Merge quoted strings: 'a' 'b' -> 'ab'; 'a''b' 'c' -> 'a''bc'
+ */
+ public static String mergeQuotedStrings(String s1, String s2) {
+ if(s1 == null || s2 == null) {
+ return null;
+ }
+
+ int len1 = s1.length();
+ int len2 = s2.length();
+
+ if(len1 == 0 || len2 == 0) {
+ return s1;
+ }
+
+ return s1.substring(0, len1 - 1) + s2.substring(1);
+ }
+
+ /**
+ * Convert String to Date
+ */
+ public static Date toDate(String s) {
+ int len = s.length();
+ if(len >= 10) {
+ int c4 = s.charAt(4);
+ int c7 = s.charAt(7);
+ // YYYY-MM-DD
+ if(c4 == '-' && c7 == '-') {
+ return Date.valueOf(s.substring(0, 10));
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Convert String to Timestamp
+ */
+ public static Timestamp toTimestamp(String s) {
+ int len = s.length();
+ if(len >= 10) {
+ int c4 = s.charAt(4);
+ int c7 = s.charAt(7);
+ // YYYY-MM-DD
+ if(c4 == '-' && c7 == '-') {
+ // Convert DB2 syntax: YYYY-MM-DD-HH.MI.SS.FFF
+ if(len > 19) {
+ if(s.charAt(10) == '-') {
+ String s2 = s.substring(0, 10) + ' ' + s.substring(11, 13) + ':' + s.substring(14, 16) + ':' +
+ s.substring(17);
+ return Timestamp.valueOf(s2);
+ }
+ }
+ else if(len == 10) {
+ s += " 00:00:00.000";
+ }
+ return Timestamp.valueOf(s);
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Compare two String values and return min or max
+ */
+ public static String minMaxString(String s1, String s2, boolean max) {
+ if(s1 == null) {
+ return s2;
+ }
+ else if(s2 == null) {
+ return s1;
+ }
+ int cmp = s1.compareTo(s2);
+ if((max && cmp < 0) || (!max && cmp > 0)) {
+ return s2;
+ }
+ return s1;
+ }
+
+ /**
+ * Compare two Int values and return min or max
+ */
+ public static Long minMaxInt(Long i1, String s, boolean max) {
+ Long i2 = null;
+ try {
+ i2 = Long.parseLong(s);
+ }
+ catch(NumberFormatException e) {}
+ if(i1 == null) {
+ return i2;
+ }
+ else if(i2 == null) {
+ return i1;
+ }
+ if((max && i1.longValue() < i2.longValue()) || (!max && i1.longValue() > i2.longValue())) {
+ return i2;
+ }
+ return i1;
+ }
+
+ /**
+ * Compare two Date values and return min or max
+ */
+ public static Date minMaxDate(Date d1, String s, boolean max) {
+ Date d2 = Utils.toDate(s);
+ if(d1 == null) {
+ return d2;
+ } else if(d2 == null) {
+ return d1;
+ }
+ if((max && d1.before(d2)) || (!max && d1.after(d2))) {
+ return d2;
+ }
+ return d1;
+ }
+
+ /**
+ * Convert String array to a string with the specified delimiter
+ */
+ public static String toString(String[] a, char del) {
+ StringBuilder s = new StringBuilder();
+ for(int i=0; i < a.length; i++) {
+ if(i > 0) {
+ s.append(del);
+ }
+ s.append(a[i]);
+ }
+ return s.toString();
+ }
+
+ /**
+ * Convert SQL datetime format string to Java SimpleDateFormat
+ */
+ public static String convertSqlDatetimeFormat(String in) {
+ StringBuilder out = new StringBuilder();
+ int len = in.length();
+ int i = 0;
+ while (i < len) {
+ if (i + 4 <= len && in.substring(i, i + 4).compareTo("YYYY") == 0) {
+ out.append("yyyy");
+ i += 4;
+ }
+ else if (i + 2 <= len && in.substring(i, i + 2).compareTo("mm") == 0) {
+ out.append("MM");
+ i += 2;
+ }
+ else if (i + 2 <= len && in.substring(i, i + 2).compareTo("DD") == 0) {
+ out.append("dd");
+ i += 2;
+ }
+ else if (i + 4 <= len && in.substring(i, i + 4).compareToIgnoreCase("HH24") == 0) {
+ out.append("HH");
+ i += 4;
+ }
+ else if (i + 2 <= len && in.substring(i, i + 2).compareToIgnoreCase("MI") == 0) {
+ out.append("mm");
+ i += 2;
+ }
+ else if (i + 2 <= len && in.substring(i, i + 2).compareTo("SS") == 0) {
+ out.append("ss");
+ i += 2;
+ }
+ else {
+ out.append(in.charAt(i));
+ i++;
+ }
+ }
+ return out.toString();
+ }
+
+ /**
+ * Get the executable directory
+ */
+ public static String getExecDir() {
+ String dir = Hplsql.class.getProtectionDomain().getCodeSource().getLocation().getPath();
+ if (dir.endsWith(".jar")) {
+ dir = dir.substring(0, dir.lastIndexOf("/") + 1);
+ }
+ return dir;
+ }
+
+ /**
+ * Format size value specified in bytes
+ */
+ public static String formatSizeInBytes(long bytes, String postfix) {
+ String out;
+ if (bytes < 1024) {
+ out = bytes + " bytes";
+ }
+ else if (bytes < 1024 * 1024) {
+ out = String.format("%.1f", ((float)bytes)/1024) + " KB";
+ }
+ else if (bytes < 1024 * 1024 * 1024) {
+ out = String.format("%.1f", ((float)bytes)/(1024 * 1024)) + " MB";
+ }
+ else {
+ out = String.format("%.1f", ((float)bytes)/(1024 * 1024 * 1024)) + " GB";
+ }
+ if (postfix != null && !postfix.isEmpty()) {
+ out += postfix;
+ }
+ return out;
+ }
+
+ public static String formatSizeInBytes(long bytes) {
+ return Utils.formatSizeInBytes(bytes, null);
+ }
+
+ /**
+ * Format bytes per second rate
+ */
+ public static String formatBytesPerSec(long bytes, long msElapsed) {
+ float bytesPerSec = ((float)bytes)/msElapsed*1000;
+ return Utils.formatSizeInBytes((long)bytesPerSec, "/sec");
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/Var.java b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
new file mode 100644
index 0000000..0a4ead2
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/Var.java
@@ -0,0 +1,430 @@
+/**
+ * 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.hive.hplsql;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Date;
+import java.sql.Timestamp;
+
+/**
+ * Variable or the result of expression
+ */
+public class Var {
+
+ // Data types
+ public enum Type {BOOL, CURSOR, DATE, DEC, FILE, IDENT, BIGINT, INTERVAL, STRING, STRINGLIST, TIMESTAMP, NULL};
+ public static Var Empty = new Var();
+ public static Var Null = new Var(Type.NULL);
+
+ public String name;
+ public Type type;
+ public Object value;
+
+ int len;
+ int scale;
+
+ public Var() {
+ type = Type.NULL;
+ }
+
+ public Var(Var var) {
+ name = var.name;
+ type = var.type;
+ value = var.value;
+ len = var.len;
+ scale = var.scale;
+ }
+
+ public Var(Long value) {
+ this.type = Type.BIGINT;
+ this.value = value;
+ }
+
+ public Var(BigDecimal value) {
+ this.type = Type.DEC;
+ this.value = value;
+ }
+
+ public Var(String name, Long value) {
+ this.type = Type.BIGINT;
+ this.name = name;
+ this.value = value;
+ }
+
+ public Var(String value) {
+ this.type = Type.STRING;
+ this.value = value;
+ }
+
+ public Var(Date value) {
+ this.type = Type.DATE;
+ this.value = value;
+ }
+
+ public Var(Timestamp value, int scale) {
+ this.type = Type.TIMESTAMP;
+ this.value = value;
+ this.scale = scale;
+ }
+
+ public Var(Interval value) {
+ this.type = Type.INTERVAL;
+ this.value = value;
+ }
+
+ public Var(ArrayList<String> value) {
+ this.type = Type.STRINGLIST;
+ this.value = value;
+ }
+
+ public Var(Boolean b) {
+ type = Type.BOOL;
+ value = b;
+ }
+
+ public Var(Type type, String name) {
+ this.type = type;
+ this.name = name;
+ }
+
+ public Var(Type type, Object value) {
+ this.type = type;
+ this.value = value;
+ }
+
+ public Var(String name, Type type, Object value) {
+ this.name = name;
+ this.type = type;
+ this.value = value;
+ }
+
+ public Var(Type type) {
+ this.type = type;
+ }
+
+ public Var(String name, String type, String len, String scale, Var def) {
+ this.name = name;
+ setType(type);
+ if (len != null) {
+ this.len = Integer.parseInt(len);
+ }
+ if (scale != null) {
+ this.scale = Integer.parseInt(scale);
+ }
+ if (def != null) {
+ cast(def);
+ }
+ }
+
+ /**
+ * Cast a new value to the variable
+ */
+ public Var cast(Var val) {
+ if (val == null || val.value == null) {
+ value = null;
+ }
+ else if (type == val.type && type == Type.STRING) {
+ cast((String)val.value);
+ }
+ else if (type == val.type) {
+ value = val.value;
+ }
+ else if (type == Type.STRING) {
+ cast(val.toString());
+ }
+ else if (type == Type.DATE) {
+ value = Utils.toDate(val.toString());
+ }
+ else if (type == Type.TIMESTAMP) {
+ value = Utils.toTimestamp(val.toString());
+ }
+ return this;
+ }
+
+ /**
+ * Cast a new string value to the variable
+ */
+ public Var cast(String val) {
+ if (type == Type.STRING) {
+ if (len != 0 ) {
+ int l = val.length();
+ if (l > len) {
+ value = val.substring(0, len);
+ return this;
+ }
+ }
+ value = val;
+ }
+ return this;
+ }
+
+ /**
+ * Set the new value
+ */
+ public void setValue(String str) {
+ if(type == Type.STRING) {
+ value = str;
+ }
+ }
+
+ public Var setValue(Long val) {
+ if (type == Type.BIGINT) {
+ value = val;
+ }
+ return this;
+ }
+
+ /**
+ * Set the new value from a result set
+ */
+ public Var setValue(ResultSet rs, ResultSetMetaData rsm, int idx) throws SQLException {
+ int type = rsm.getColumnType(idx);
+ if (type == java.sql.Types.CHAR || type == java.sql.Types.VARCHAR) {
+ cast(new Var(rs.getString(idx)));
+ }
+ else if (type == java.sql.Types.INTEGER || type == java.sql.Types.BIGINT) {
+ cast(new Var(new Long(rs.getLong(idx))));
+ }
+ return this;
+ }
+
+ /**
+ * Set the data type from string representation
+ */
+ void setType(String type) {
+ this.type = defineType(type);
+ }
+
+ /**
+ * Set the data type from JDBC type code
+ */
+ void setType(int type) {
+ this.type = defineType(type);
+ }
+
+ /**
+ * Define the data type from string representation
+ */
+ public static Type defineType(String type) {
+ if (type == null) {
+ return Type.NULL;
+ }
+ else if (type.equalsIgnoreCase("INT") || type.equalsIgnoreCase("INTEGER")) {
+ return Type.BIGINT;
+ }
+ else if (type.equalsIgnoreCase("CHAR") || type.equalsIgnoreCase("VARCHAR") || type.equalsIgnoreCase("STRING")) {
+ return Type.STRING;
+ }
+ else if (type.equalsIgnoreCase("DATE")) {
+ return Type.DATE;
+ }
+ else if (type.equalsIgnoreCase("TIMESTAMP")) {
+ return Type.TIMESTAMP;
+ }
+ else if (type.equalsIgnoreCase("UTL_FILE.FILE_TYPE")) {
+ return Type.FILE;
+ }
+ return Type.NULL;
+ }
+
+ /**
+ * Define the data type from JDBC type code
+ */
+ public static Type defineType(int type) {
+ if (type == java.sql.Types.CHAR || type == java.sql.Types.VARCHAR) {
+ return Type.STRING;
+ }
+ else if (type == java.sql.Types.INTEGER || type == java.sql.Types.BIGINT) {
+ return Type.BIGINT;
+ }
+ return Type.NULL;
+ }
+
+ /**
+ * Remove value
+ */
+ public void removeValue() {
+ type = Type.NULL;
+ name = null;
+ value = null;
+ len = 0;
+ scale = 0;
+ }
+
+ /*
+ * Compare values
+ */
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ else if (obj == null || this.value == null) {
+ return false;
+ }
+ else if (getClass() != obj.getClass()) {
+ return false;
+ }
+
+ Var var = (Var)obj;
+ if (type == Type.BIGINT && var.type == Type.BIGINT &&
+ ((Long)value).longValue() == ((Long)var.value).longValue()) {
+ return true;
+ }
+ else if (type == Type.STRING && var.type == Type.STRING &&
+ ((String)value).equals((String)var.value)) {
+ return true;
+ }
+ return false;
+ }
+
+ /*
+ * Compare values
+ */
+ public int compareTo(Var v) {
+ if (this == v) {
+ return 0;
+ }
+ else if (v == null) {
+ return -1;
+ }
+ else if (type == Type.BIGINT && v.type == Type.BIGINT) {
+ return ((Long)value).compareTo((Long)v.value);
+ }
+ else if (type == Type.STRING && v.type == Type.STRING) {
+ return ((String)value).compareTo((String)v.value);
+ }
+ return -1;
+ }
+
+ /**
+ * Increment an integer value
+ */
+ public Var increment(Long i) {
+ if (type == Type.BIGINT) {
+ value = new Long(((Long)value).longValue() + i);
+ }
+ return this;
+ }
+
+ /**
+ * Decrement an integer value
+ */
+ public Var decrement(Long i) {
+ if (type == Type.BIGINT) {
+ value = new Long(((Long)value).longValue() - i);
+ }
+ return this;
+ }
+
+ /**
+ * Return an integer value
+ */
+ public int intValue() {
+ if (type == Type.BIGINT) {
+ return ((Long)value).intValue();
+ }
+ return -1;
+ }
+
+ /**
+ * Return true/false for BOOL type
+ */
+ public boolean isTrue() {
+ if(type == Type.BOOL && value != null) {
+ return ((Boolean)value).booleanValue();
+ }
+ return false;
+ }
+
+ /**
+ * Check if the variable contains NULL
+ */
+ public boolean isNull() {
+ if (type == Type.NULL || value == null) {
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Convert value to String
+ */
+ @Override
+ public String toString() {
+ if (type == Type.IDENT) {
+ return name;
+ }
+ else if (value == null) {
+ return null;
+ }
+ else if (type == Type.BIGINT) {
+ return ((Long)value).toString();
+ }
+ else if (type == Type.STRING) {
+ return (String)value;
+ }
+ else if (type == Type.DATE) {
+ return ((Date)value).toString();
+ }
+ else if (type == Type.TIMESTAMP) {
+ int len = 19;
+ String t = ((Timestamp)value).toString(); // .0 returned if the fractional part not set
+ if (scale > 0) {
+ len += scale + 1;
+ }
+ if (t.length() > len) {
+ t = t.substring(0, len);
+ }
+ return t;
+ }
+ return value.toString();
+ }
+
+ /**
+ * Convert value to SQL string - string literals are quoted and escaped, ab'c -> 'ab''c'
+ */
+ public String toSqlString() {
+ if (value == null) {
+ return "NULL";
+ }
+ else if (type == Type.STRING) {
+ return Utils.quoteString((String)value);
+ }
+ return toString();
+ }
+
+ /**
+ * Set variable name
+ */
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ /**
+ * Get variable name
+ */
+ public String getName() {
+ return name;
+ }
+}
[21/25] hive git commit: HIVE-11152 : Swapping join inputs in
ASTConverter (Jesus Camacho Rodriguez via John Pullokkaran)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join33.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join33.q.out b/ql/src/test/results/clientpositive/spark/join33.q.out
index d7383fb..4ae9dc6 100644
--- a/ql/src/test/results/clientpositive/spark/join33.q.out
+++ b/ql/src/test/results/clientpositive/spark/join33.q.out
@@ -110,25 +110,25 @@ STAGE PLANS:
Spark
#### A masked pattern was here ####
Vertices:
- Map 1
+ Map 2
Map Operator Tree:
TableScan
- alias: y
- Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ alias: x
+ Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: key is not null (type: boolean)
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ predicate: (value is not null and key is not null) (type: boolean)
+ Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
0 _col0 (type: string)
- 1 _col3 (type: string)
- Position of Big Table: 1
+ 1 _col1 (type: string)
+ Position of Big Table: 0
Local Work:
Map Reduce Local Work
Path -> Alias:
@@ -136,7 +136,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src
+ base file name: src1
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -146,14 +146,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -166,38 +166,38 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src
+ name default.src1
numFiles 1
- numRows 500
- rawDataSize 5312
- serialization.ddl struct src { string key, string value}
+ numRows 25
+ rawDataSize 191
+ serialization.ddl struct src1 { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 5812
+ totalSize 216
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src
- name: default.src
+ name: default.src1
+ name: default.src1
Truncated Path -> Alias:
- /src [y]
+ /src1 [x]
Map 3
Map Operator Tree:
TableScan
- alias: x
- Statistics: Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ alias: y
+ Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Filter Operator
isSamplingPred: false
- predicate: (value is not null and key is not null) (type: boolean)
- Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+ predicate: key is not null (type: boolean)
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
Spark HashTable Sink Operator
keys:
- 0 _col0 (type: string)
- 1 _col1 (type: string)
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
Position of Big Table: 0
Local Work:
Map Reduce Local Work
@@ -206,7 +206,7 @@ STAGE PLANS:
Path -> Partition:
#### A masked pattern was here ####
Partition
- base file name: src1
+ base file name: src
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
properties:
@@ -216,14 +216,14 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
@@ -236,26 +236,26 @@ STAGE PLANS:
columns.comments 'default','default'
columns.types string:string
#### A masked pattern was here ####
- name default.src1
+ name default.src
numFiles 1
- numRows 25
- rawDataSize 191
- serialization.ddl struct src1 { string key, string value}
+ numRows 500
+ rawDataSize 5312
+ serialization.ddl struct src { string key, string value}
serialization.format 1
serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- totalSize 216
+ totalSize 5812
#### A masked pattern was here ####
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- name: default.src1
- name: default.src1
+ name: default.src
+ name: default.src
Truncated Path -> Alias:
- /src1 [x]
+ /src [y]
Stage: Stage-1
Spark
#### A masked pattern was here ####
Vertices:
- Map 2
+ Map 1
Map Operator Tree:
TableScan
alias: z
@@ -277,22 +277,22 @@ STAGE PLANS:
1 _col1 (type: string)
outputColumnNames: _col0, _col3
input vertices:
- 1 Map 3
+ 1 Map 2
Position of Big Table: 0
Statistics: Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
Map Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col1, _col2, _col5
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col3, _col6
input vertices:
- 0 Map 1
- Position of Big Table: 1
+ 1 Map 3
+ Position of Big Table: 0
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col5 (type: string), _col2 (type: string), _col1 (type: string)
+ expressions: _col3 (type: string), _col0 (type: string), _col6 (type: string)
outputColumnNames: _col0, _col1, _col2
Statistics: Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
File Output Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out b/ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out
index f34153d..a68321f 100644
--- a/ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_alt_syntax.q.out
@@ -394,9 +394,9 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2)
- Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2)
+ Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -405,35 +405,17 @@ STAGE PLANS:
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: p_partkey is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ predicate: (p_partkey is not null and p_name is not null) (type: boolean)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string)
- Map 3
- Map Operator Tree:
- TableScan
- alias: p1
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: p_name is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: p_name (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: string)
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ key expressions: _col0 (type: int), _col1 (type: string)
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Map 5
Map Operator Tree:
TableScan
@@ -451,76 +433,94 @@ STAGE PLANS:
sort order: ++
Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: p1
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: p_name is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_name (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Map 7
Map Operator Tree:
TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (p_partkey is not null and p_name is not null) (type: boolean)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ predicate: p_partkey is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int), _col1 (type: string)
- sort order: ++
- Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string)
Reducer 2
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: int)
- 1 _col1 (type: int)
- outputColumnNames: _col1, _col2, _col4, _col6
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col4 (type: string), _col6 (type: string), _col2 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 4
+ 0 _col0 (type: int), _col1 (type: string)
+ 1 _col0 (type: int), _col1 (type: string)
+ outputColumnNames: _col0, _col1, _col3
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col3 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col3 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string)
+ Reducer 3
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col4
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col1 (type: int)
+ key expressions: _col0 (type: int)
sort order: +
- Map-reduce partition columns: _col1 (type: int)
+ Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string), _col2 (type: string), _col4 (type: string)
- Reducer 6
+ value expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string)
+ Reducer 4
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: int), _col1 (type: string)
- 1 _col0 (type: int), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col3
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col3 (type: string)
- sort order: +
- Map-reduce partition columns: _col3 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string)
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col1, _col3, _col4, _col6
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
@@ -546,9 +546,9 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2)
- Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2)
+ Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -557,35 +557,17 @@ STAGE PLANS:
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: p_partkey is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ predicate: (p_name is not null and p_partkey is not null) (type: boolean)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string)
- Map 3
- Map Operator Tree:
- TableScan
- alias: p1
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: p_name is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: p_name (type: string)
- outputColumnNames: _col0
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: string)
- sort order: +
- Map-reduce partition columns: _col0 (type: string)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ key expressions: _col1 (type: string), _col0 (type: int)
+ sort order: ++
+ Map-reduce partition columns: _col1 (type: string), _col0 (type: int)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Map 5
Map Operator Tree:
TableScan
@@ -603,76 +585,94 @@ STAGE PLANS:
sort order: ++
Map-reduce partition columns: _col1 (type: string), _col0 (type: int)
Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: p1
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: p_name is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_name (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Map 7
Map Operator Tree:
TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (p_name is not null and p_partkey is not null) (type: boolean)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ predicate: p_partkey is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string)
outputColumnNames: _col0, _col1
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col1 (type: string), _col0 (type: int)
- sort order: ++
- Map-reduce partition columns: _col1 (type: string), _col0 (type: int)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string)
Reducer 2
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: int)
- 1 _col1 (type: int)
- outputColumnNames: _col1, _col2, _col4, _col6
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col4 (type: string), _col6 (type: string), _col2 (type: string), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 4
+ 0 _col1 (type: string), _col0 (type: int)
+ 1 _col1 (type: string), _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col3
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col3 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col3 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string)
+ Reducer 3
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: string)
- 1 _col3 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col4
+ 0 _col3 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col1, _col3, _col4
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col1 (type: int)
+ key expressions: _col0 (type: int)
sort order: +
- Map-reduce partition columns: _col1 (type: int)
+ Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: string), _col2 (type: string), _col4 (type: string)
- Reducer 6
+ value expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string)
+ Reducer 4
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col1 (type: string), _col0 (type: int)
- 1 _col1 (type: string), _col0 (type: int)
- outputColumnNames: _col0, _col1, _col3
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col3 (type: string)
- sort order: +
- Map-reduce partition columns: _col3 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string)
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col1, _col3, _col4, _col6
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: string), _col3 (type: string), _col4 (type: string), _col6 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out
index aa84423..fe2da0e 100644
--- a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_1.q.out
@@ -301,7 +301,7 @@ STAGE PLANS:
Processor Tree:
ListSink
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product
+Warning: Shuffle Join JOIN[15][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Work 'Reducer 3' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3 on p2.p_partkey = 1 and p3.p_name = p2.p_name
PREHOOK: type: QUERY
@@ -316,8 +316,8 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -325,19 +325,6 @@ STAGE PLANS:
TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- sort order:
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- Map 3
- Map Operator Tree:
- TableScan
- alias: p1
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: p_name is not null (type: boolean)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
@@ -351,7 +338,7 @@ STAGE PLANS:
Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- Map 5
+ Map 4
Map Operator Tree:
TableScan
alias: p1
@@ -369,18 +356,45 @@ STAGE PLANS:
Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: p1
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Reducer 2
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
+ 0 _col1 (type: string)
+ 1 _col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
0
1
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), 1 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), 1 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -390,24 +404,6 @@ STAGE PLANS:
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 4
- Reduce Operator Tree:
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col1 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- sort order:
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out
index 26e05ac..5f572b0 100644
--- a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_2.q.out
@@ -150,9 +150,9 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2)
- Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2)
+ Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -161,36 +161,18 @@ STAGE PLANS:
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: p_partkey is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- Map 3
- Map Operator Tree:
- TableScan
- alias: p1
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: p_name is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ predicate: (p_partkey is not null and p_name is not null) (type: boolean)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col1 (type: string)
- sort order: +
- Map-reduce partition columns: _col1 (type: string)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ key expressions: _col0 (type: int), _col1 (type: string)
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Map 5
Map Operator Tree:
TableScan
@@ -209,77 +191,91 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: p1
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: p_name is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Map 7
Map Operator Tree:
TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (p_partkey is not null and p_name is not null) (type: boolean)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ predicate: p_partkey is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int), _col1 (type: string)
- sort order: ++
- Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Reducer 2
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: int)
- 1 _col9 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col27 (type: int), _col28 (type: string), _col29 (type: string), _col30 (type: string), _col31 (type: string), _col32 (type: int), _col33 (type: string), _col34 (type: double), _col35 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 4
+ 0 _col0 (type: int), _col1 (type: string)
+ 1 _col0 (type: int), _col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col10 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ Reducer 3
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col1 (type: string)
- 1 _col10 (type: string)
+ 0 _col10 (type: string)
+ 1 _col1 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col9 (type: int)
+ key expressions: _col0 (type: int)
sort order: +
- Map-reduce partition columns: _col9 (type: int)
+ Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
- Reducer 6
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
+ Reducer 4
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: int), _col1 (type: string)
- 1 _col0 (type: int), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col10 (type: string)
- sort order: +
- Map-reduce partition columns: _col10 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out
index 4607309..c0bdd80 100644
--- a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_3.q.out
@@ -307,7 +307,7 @@ STAGE PLANS:
Processor Tree:
ListSink
-Warning: Shuffle Join JOIN[16][tables = [$hdt$_0, $hdt$_1]] in Work 'Reducer 2' is a cross product
+Warning: Shuffle Join JOIN[15][tables = [$hdt$_1, $hdt$_2, $hdt$_0]] in Work 'Reducer 3' is a cross product
PREHOOK: query: explain select *
from part p1 join part p2 join part p3
where p2.p_partkey = 1 and p3.p_name = p2.p_name
@@ -324,8 +324,8 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 1), Reducer 4 (PARTITION-LEVEL SORT, 1)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 5 (PARTITION-LEVEL SORT, 1), Reducer 2 (PARTITION-LEVEL SORT, 1)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -333,19 +333,6 @@ STAGE PLANS:
TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- sort order:
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- Map 3
- Map Operator Tree:
- TableScan
- alias: p1
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: p_name is not null (type: boolean)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
@@ -359,7 +346,7 @@ STAGE PLANS:
Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- Map 5
+ Map 4
Map Operator Tree:
TableScan
alias: p1
@@ -377,18 +364,45 @@ STAGE PLANS:
Map-reduce partition columns: _col1 (type: string)
Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Map 5
+ Map Operator Tree:
+ TableScan
+ alias: p1
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Reducer 2
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
+ 0 _col1 (type: string)
+ 1 _col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ sort order:
+ Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ Reducer 3
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
0
1
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), 1 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), 1 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 28 Data size: 3461 Basic stats: COMPLETE Column stats: NONE
File Output Operator
@@ -398,24 +412,6 @@ STAGE PLANS:
input format: org.apache.hadoop.mapred.TextInputFormat
output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 4
- Reduce Operator Tree:
- Join Operator
- condition map:
- Inner Join 0 to 1
- keys:
- 0 _col1 (type: string)
- 1 _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col0 (type: int), _col1 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- sort order:
- Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
Stage: Stage-0
Fetch Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out
index c821fe4..ddc6812 100644
--- a/ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_cond_pushdown_4.q.out
@@ -154,9 +154,9 @@ STAGE PLANS:
Stage: Stage-1
Spark
Edges:
- Reducer 6 <- Map 5 (PARTITION-LEVEL SORT, 2), Map 7 (PARTITION-LEVEL SORT, 2)
- Reducer 4 <- Map 3 (PARTITION-LEVEL SORT, 2), Reducer 6 (PARTITION-LEVEL SORT, 2)
- Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Reducer 4 (PARTITION-LEVEL SORT, 2)
+ Reducer 2 <- Map 1 (PARTITION-LEVEL SORT, 2), Map 5 (PARTITION-LEVEL SORT, 2)
+ Reducer 3 <- Map 6 (PARTITION-LEVEL SORT, 2), Reducer 2 (PARTITION-LEVEL SORT, 2)
+ Reducer 4 <- Map 7 (PARTITION-LEVEL SORT, 2), Reducer 3 (PARTITION-LEVEL SORT, 2)
#### A masked pattern was here ####
Vertices:
Map 1
@@ -165,36 +165,18 @@ STAGE PLANS:
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: p_partkey is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col0 (type: int)
- sort order: +
- Map-reduce partition columns: _col0 (type: int)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- Map 3
- Map Operator Tree:
- TableScan
- alias: p1
- Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
- Filter Operator
- predicate: p_name is not null (type: boolean)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ predicate: (p_partkey is not null and p_name is not null) (type: boolean)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col1 (type: string)
- sort order: +
- Map-reduce partition columns: _col1 (type: string)
- Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ key expressions: _col0 (type: int), _col1 (type: string)
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
+ Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Map 5
Map Operator Tree:
TableScan
@@ -213,77 +195,91 @@ STAGE PLANS:
Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ Map 6
+ Map Operator Tree:
+ TableScan
+ alias: p1
+ Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator
+ predicate: p_name is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Map 7
Map Operator Tree:
TableScan
alias: p1
Statistics: Num rows: 26 Data size: 3147 Basic stats: COMPLETE Column stats: NONE
Filter Operator
- predicate: (p_partkey is not null and p_name is not null) (type: boolean)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ predicate: p_partkey is not null (type: boolean)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: p_partkey (type: int), p_name (type: string), p_mfgr (type: string), p_brand (type: string), p_type (type: string), p_size (type: int), p_container (type: string), p_retailprice (type: double), p_comment (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col0 (type: int), _col1 (type: string)
- sort order: ++
- Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
- Statistics: Num rows: 7 Data size: 847 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
+ key expressions: _col0 (type: int)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: int)
+ Statistics: Num rows: 13 Data size: 1573 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
Reducer 2
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: int)
- 1 _col9 (type: int)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- Select Operator
- expressions: _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string), _col27 (type: int), _col28 (type: string), _col29 (type: string), _col30 (type: string), _col31 (type: string), _col32 (type: int), _col33 (type: string), _col34 (type: double), _col35 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- File Output Operator
- compressed: false
- Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
- table:
- input format: org.apache.hadoop.mapred.TextInputFormat
- output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
- serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- Reducer 4
+ 0 _col0 (type: int), _col1 (type: string)
+ 1 _col0 (type: int), _col1 (type: string)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col10 (type: string)
+ sort order: +
+ Map-reduce partition columns: _col10 (type: string)
+ Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
+ value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ Reducer 3
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col1 (type: string)
- 1 _col10 (type: string)
+ 0 _col10 (type: string)
+ 1 _col1 (type: string)
outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
Reduce Output Operator
- key expressions: _col9 (type: int)
+ key expressions: _col0 (type: int)
sort order: +
- Map-reduce partition columns: _col9 (type: int)
+ Map-reduce partition columns: _col0 (type: int)
Statistics: Num rows: 14 Data size: 1730 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
- Reducer 6
+ value expressions: _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col10 (type: string), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string), _col18 (type: int), _col19 (type: string), _col20 (type: string), _col21 (type: string), _col22 (type: string), _col23 (type: int), _col24 (type: string), _col25 (type: double), _col26 (type: string)
+ Reducer 4
Reduce Operator Tree:
Join Operator
condition map:
Inner Join 0 to 1
keys:
- 0 _col0 (type: int), _col1 (type: string)
- 1 _col0 (type: int), _col1 (type: string)
- outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- Reduce Output Operator
- key expressions: _col10 (type: string)
- sort order: +
- Map-reduce partition columns: _col10 (type: string)
- Statistics: Num rows: 7 Data size: 931 Basic stats: COMPLETE Column stats: NONE
- value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: int), _col6 (type: string), _col7 (type: double), _col8 (type: string), _col9 (type: int), _col11 (type: string), _col12 (type: string), _col13 (type: string), _col14 (type: int), _col15 (type: string), _col16 (type: double), _col17 (type: string)
+ 0 _col0 (type: int)
+ 1 _col0 (type: int)
+ outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 15 Data size: 1903 Basic stats: COMPLETE Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
Stage: Stage-0
Fetch Operator
[19/25] hive git commit: HIVE-11152 : Swapping join inputs in
ASTConverter (Jesus Camacho Rodriguez via John Pullokkaran)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/6eaa32c8/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
index c063421..71d8f41 100644
--- a/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
+++ b/ql/src/test/results/clientpositive/tez/explainuser_2.q.out
@@ -181,83 +181,83 @@ POSTHOOK: type: QUERY
Plan optimized by CBO.
Vertex dependency in root stage
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
-Reducer 4 <- Map 3 (SIMPLE_EDGE), Map 5 (SIMPLE_EDGE)
+Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 4 (SIMPLE_EDGE)
+Reducer 3 <- Map 5 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
- Reducer 2
- File Output Operator [FS_19]
+ Reducer 3
+ File Output Operator [FS_18]
compressed:false
Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
- Select Operator [SEL_18]
+ Select Operator [SEL_17]
outputColumnNames:["_col0","_col1","_col2"]
Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
- Merge Join Operator [MERGEJOIN_29]
+ Merge Join Operator [MERGEJOIN_28]
| condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col3 (type: string)","0":"_col0 (type: string)"}
- | outputColumnNames:["_col1","_col2","_col5"]
+ | keys:{"1":"_col0 (type: string)","0":"_col3 (type: string)"}
+ | outputColumnNames:["_col0","_col3","_col6"]
| Statistics:Num rows: 302 Data size: 3213 Basic stats: COMPLETE Column stats: NONE
- |<-Map 1 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_14]
+ |<-Map 5 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_15]
| key expressions:_col0 (type: string)
| Map-reduce partition columns:_col0 (type: string)
| sort order:+
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
| value expressions:_col1 (type: string)
- | Select Operator [SEL_1]
+ | Select Operator [SEL_6]
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_25]
+ | Filter Operator [FIL_26]
| predicate:key is not null (type: boolean)
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_0]
+ | TableScan [TS_5]
| alias:y
| Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 4 [SIMPLE_EDGE]
- Reduce Output Operator [RS_16]
+ |<-Reducer 2 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_13]
key expressions:_col3 (type: string)
Map-reduce partition columns:_col3 (type: string)
sort order:+
Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
value expressions:_col0 (type: string)
- Merge Join Operator [MERGEJOIN_28]
+ Merge Join Operator [MERGEJOIN_27]
| condition map:[{"":"Inner Join 0 to 1"}]
| keys:{"1":"_col1 (type: string)","0":"_col0 (type: string)"}
| outputColumnNames:["_col0","_col3"]
| Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- |<-Map 3 [SIMPLE_EDGE]
+ |<-Map 1 [SIMPLE_EDGE]
| Reduce Output Operator [RS_8]
| key expressions:_col0 (type: string)
| Map-reduce partition columns:_col0 (type: string)
| sort order:+
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_4]
+ | Select Operator [SEL_2]
| outputColumnNames:["_col0"]
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_26]
+ | Filter Operator [FIL_24]
| predicate:value is not null (type: boolean)
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_2]
+ | TableScan [TS_0]
| alias:z
| Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- |<-Map 5 [SIMPLE_EDGE]
+ |<-Map 4 [SIMPLE_EDGE]
Reduce Output Operator [RS_10]
key expressions:_col1 (type: string)
Map-reduce partition columns:_col1 (type: string)
sort order:+
Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
value expressions:_col0 (type: string)
- Select Operator [SEL_6]
+ Select Operator [SEL_4]
outputColumnNames:["_col0","_col1"]
Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- Filter Operator [FIL_27]
+ Filter Operator [FIL_25]
predicate:(value is not null and key is not null) (type: boolean)
Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- TableScan [TS_5]
+ TableScan [TS_3]
alias:x
Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
@@ -320,260 +320,260 @@ POSTHOOK: type: QUERY
Plan optimized by CBO.
Vertex dependency in root stage
-Reducer 11 <- Map 10 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
-Reducer 13 <- Map 12 (SIMPLE_EDGE), Map 14 (SIMPLE_EDGE)
+Reducer 10 <- Map 14 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 11 <- Reducer 10 (SIMPLE_EDGE), Reducer 16 (SIMPLE_EDGE)
Reducer 16 <- Map 15 (SIMPLE_EDGE), Map 17 (SIMPLE_EDGE)
Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 6 (SIMPLE_EDGE)
-Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
+Reducer 3 <- Reducer 11 (SIMPLE_EDGE), Reducer 2 (SIMPLE_EDGE)
Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
Reducer 5 <- Reducer 4 (SIMPLE_EDGE)
-Reducer 8 <- Map 7 (SIMPLE_EDGE), Reducer 11 (SIMPLE_EDGE)
-Reducer 9 <- Reducer 16 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
+Reducer 8 <- Map 12 (SIMPLE_EDGE), Map 7 (SIMPLE_EDGE)
+Reducer 9 <- Map 13 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:100
Stage-1
Reducer 5
- File Output Operator [FS_71]
+ File Output Operator [FS_69]
compressed:false
Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
- Limit [LIM_70]
+ Limit [LIM_68]
Number of rows:100
Statistics:Num rows: 100 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
- Select Operator [SEL_69]
+ Select Operator [SEL_67]
| outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
| Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
|<-Reducer 4 [SIMPLE_EDGE]
- Reduce Output Operator [RS_68]
+ Reduce Output Operator [RS_66]
key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
sort order:+++
Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
- Group By Operator [GBY_66]
+ Group By Operator [GBY_64]
| aggregations:["count(VALUE._col0)","count(VALUE._col1)","count(VALUE._col2)"]
| keys:KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
| outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
| Statistics:Num rows: 402 Data size: 4276 Basic stats: COMPLETE Column stats: NONE
|<-Reducer 3 [SIMPLE_EDGE]
- Reduce Output Operator [RS_65]
+ Reduce Output Operator [RS_63]
key expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string)
Map-reduce partition columns:_col0 (type: string), _col1 (type: string), _col2 (type: string)
sort order:+++
Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
value expressions:_col3 (type: bigint), _col4 (type: bigint), _col5 (type: bigint)
- Group By Operator [GBY_64]
+ Group By Operator [GBY_62]
aggregations:["count(_col3)","count(_col4)","count(_col5)"]
keys:_col0 (type: string), _col1 (type: string), _col2 (type: string)
outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
- Select Operator [SEL_62]
+ Select Operator [SEL_60]
outputColumnNames:["_col0","_col1","_col2","_col3","_col4","_col5"]
Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
- Merge Join Operator [MERGEJOIN_113]
+ Merge Join Operator [MERGEJOIN_111]
| condition map:[{"":"Inner Join 0 to 1"}]
| keys:{"1":"_col15 (type: string), _col17 (type: string)","0":"_col1 (type: string), _col3 (type: string)"}
| outputColumnNames:["_col2","_col3","_col12","_col13","_col20","_col21"]
| Statistics:Num rows: 804 Data size: 8552 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 2 [SIMPLE_EDGE]
+ |<-Reducer 11 [SIMPLE_EDGE]
| Reduce Output Operator [RS_58]
- | key expressions:_col1 (type: string), _col3 (type: string)
- | Map-reduce partition columns:_col1 (type: string), _col3 (type: string)
+ | key expressions:_col15 (type: string), _col17 (type: string)
+ | Map-reduce partition columns:_col15 (type: string), _col17 (type: string)
| sort order:++
- | Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
- | value expressions:_col2 (type: string)
- | Merge Join Operator [MERGEJOIN_107]
- | | condition map:[{"":"Inner Join 0 to 1"}]
- | | keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
- | | outputColumnNames:["_col1","_col2","_col3"]
- | | Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 1 [SIMPLE_EDGE]
- | | Reduce Output Operator [RS_53]
- | | key expressions:_col0 (type: string)
- | | Map-reduce partition columns:_col0 (type: string)
- | | sort order:+
- | | Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
- | | value expressions:_col1 (type: string), _col2 (type: string), _col3 (type: string)
- | | Select Operator [SEL_1]
- | | outputColumnNames:["_col0","_col1","_col2","_col3"]
- | | Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
- | | Filter Operator [FIL_99]
- | | predicate:((k1 is not null and v2 is not null) and v3 is not null) (type: boolean)
- | | Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
- | | TableScan [TS_0]
- | | alias:cs
- | | Statistics:Num rows: 170 Data size: 5890 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 6 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_55]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_4]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_100]
- | predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_2]
- | alias:d1
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 9 [SIMPLE_EDGE]
- Reduce Output Operator [RS_60]
- key expressions:_col15 (type: string), _col17 (type: string)
- Map-reduce partition columns:_col15 (type: string), _col17 (type: string)
+ | Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+ | value expressions:_col6 (type: string), _col7 (type: string), _col14 (type: string)
+ | Select Operator [SEL_49]
+ | outputColumnNames:["_col14","_col15","_col17","_col6","_col7"]
+ | Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+ | Merge Join Operator [MERGEJOIN_110]
+ | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | keys:{"1":"_col2 (type: string), _col4 (type: string)","0":"_col4 (type: string), _col6 (type: string)"}
+ | | outputColumnNames:["_col2","_col3","_col14","_col15","_col17"]
+ | | Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
+ | |<-Reducer 10 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_45]
+ | | key expressions:_col4 (type: string), _col6 (type: string)
+ | | Map-reduce partition columns:_col4 (type: string), _col6 (type: string)
+ | | sort order:++
+ | | Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
+ | | value expressions:_col2 (type: string), _col3 (type: string)
+ | | Merge Join Operator [MERGEJOIN_108]
+ | | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | | keys:{"1":"_col1 (type: string)","0":"_col3 (type: string)"}
+ | | | outputColumnNames:["_col2","_col3","_col4","_col6"]
+ | | | Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
+ | | |<-Map 14 [SIMPLE_EDGE]
+ | | | Reduce Output Operator [RS_42]
+ | | | key expressions:_col1 (type: string)
+ | | | Map-reduce partition columns:_col1 (type: string)
+ | | | sort order:+
+ | | | Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+ | | | Select Operator [SEL_16]
+ | | | outputColumnNames:["_col1"]
+ | | | Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+ | | | Filter Operator [FIL_102]
+ | | | predicate:((key = 'src1key') and value is not null) (type: boolean)
+ | | | Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
+ | | | TableScan [TS_14]
+ | | | alias:src1
+ | | | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
+ | | |<-Reducer 9 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_40]
+ | | key expressions:_col3 (type: string)
+ | | Map-reduce partition columns:_col3 (type: string)
+ | | sort order:+
+ | | Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+ | | value expressions:_col2 (type: string), _col4 (type: string), _col6 (type: string)
+ | | Merge Join Operator [MERGEJOIN_107]
+ | | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | | keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+ | | | outputColumnNames:["_col2","_col3","_col4","_col6"]
+ | | | Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
+ | | |<-Map 13 [SIMPLE_EDGE]
+ | | | Reduce Output Operator [RS_37]
+ | | | key expressions:_col0 (type: string)
+ | | | Map-reduce partition columns:_col0 (type: string)
+ | | | sort order:+
+ | | | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ | | | Select Operator [SEL_13]
+ | | | outputColumnNames:["_col0"]
+ | | | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ | | | Filter Operator [FIL_101]
+ | | | predicate:((value = 'd1value') and key is not null) (type: boolean)
+ | | | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ | | | TableScan [TS_11]
+ | | | alias:d1
+ | | | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ | | |<-Reducer 8 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_35]
+ | | key expressions:_col2 (type: string)
+ | | Map-reduce partition columns:_col2 (type: string)
+ | | sort order:+
+ | | Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ | | value expressions:_col3 (type: string), _col4 (type: string), _col6 (type: string)
+ | | Merge Join Operator [MERGEJOIN_106]
+ | | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | | keys:{"1":"_col3 (type: string)","0":"_col1 (type: string)"}
+ | | | outputColumnNames:["_col2","_col3","_col4","_col6"]
+ | | | Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
+ | | |<-Map 12 [SIMPLE_EDGE]
+ | | | Reduce Output Operator [RS_32]
+ | | | key expressions:_col3 (type: string)
+ | | | Map-reduce partition columns:_col3 (type: string)
+ | | | sort order:+
+ | | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+ | | | value expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string), _col4 (type: string)
+ | | | Select Operator [SEL_10]
+ | | | outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
+ | | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+ | | | Filter Operator [FIL_100]
+ | | | predicate:((((((v3 = 'ssv3') and v2 is not null) and k1 is not null) and v1 is not null) and k2 is not null) and k3 is not null) (type: boolean)
+ | | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+ | | | TableScan [TS_8]
+ | | | alias:ss
+ | | | Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
+ | | |<-Map 7 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_30]
+ | | key expressions:_col1 (type: string)
+ | | Map-reduce partition columns:_col1 (type: string)
+ | | sort order:+
+ | | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ | | Select Operator [SEL_7]
+ | | outputColumnNames:["_col1"]
+ | | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ | | Filter Operator [FIL_99]
+ | | predicate:((key = 'srcpartkey') and value is not null) (type: boolean)
+ | | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ | | TableScan [TS_5]
+ | | alias:srcpart
+ | | Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
+ | |<-Reducer 16 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_47]
+ | key expressions:_col2 (type: string), _col4 (type: string)
+ | Map-reduce partition columns:_col2 (type: string), _col4 (type: string)
+ | sort order:++
+ | Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+ | value expressions:_col3 (type: string), _col5 (type: string)
+ | Merge Join Operator [MERGEJOIN_109]
+ | | condition map:[{"":"Inner Join 0 to 1"}]
+ | | keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+ | | outputColumnNames:["_col2","_col3","_col4","_col5"]
+ | | Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+ | |<-Map 15 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_24]
+ | | key expressions:_col0 (type: string)
+ | | Map-reduce partition columns:_col0 (type: string)
+ | | sort order:+
+ | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+ | | value expressions:_col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
+ | | Select Operator [SEL_19]
+ | | outputColumnNames:["_col0","_col2","_col3","_col4","_col5"]
+ | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+ | | Filter Operator [FIL_103]
+ | | predicate:((((((v1 = 'srv1') and k1 is not null) and k2 is not null) and k3 is not null) and v2 is not null) and v3 is not null) (type: boolean)
+ | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
+ | | TableScan [TS_17]
+ | | alias:sr
+ | | Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
+ | |<-Map 17 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_26]
+ | key expressions:_col0 (type: string)
+ | Map-reduce partition columns:_col0 (type: string)
+ | sort order:+
+ | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ | Select Operator [SEL_22]
+ | outputColumnNames:["_col0"]
+ | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ | Filter Operator [FIL_104]
+ | predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
+ | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ | TableScan [TS_20]
+ | alias:d1
+ | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
+ |<-Reducer 2 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_56]
+ key expressions:_col1 (type: string), _col3 (type: string)
+ Map-reduce partition columns:_col1 (type: string), _col3 (type: string)
sort order:++
- Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
- value expressions:_col6 (type: string), _col7 (type: string), _col14 (type: string)
- Select Operator [SEL_51]
- outputColumnNames:["_col14","_col15","_col17","_col6","_col7"]
- Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
- Merge Join Operator [MERGEJOIN_112]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col2 (type: string), _col4 (type: string)","0":"_col8 (type: string), _col10 (type: string)"}
- | outputColumnNames:["_col6","_col7","_col14","_col15","_col17"]
- | Statistics:Num rows: 731 Data size: 7775 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 16 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_49]
- | key expressions:_col2 (type: string), _col4 (type: string)
- | Map-reduce partition columns:_col2 (type: string), _col4 (type: string)
- | sort order:++
- | Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
- | value expressions:_col3 (type: string), _col5 (type: string)
- | Merge Join Operator [MERGEJOIN_111]
- | | condition map:[{"":"Inner Join 0 to 1"}]
- | | keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
- | | outputColumnNames:["_col2","_col3","_col4","_col5"]
- | | Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 15 [SIMPLE_EDGE]
- | | Reduce Output Operator [RS_36]
- | | key expressions:_col0 (type: string)
- | | Map-reduce partition columns:_col0 (type: string)
- | | sort order:+
- | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
- | | value expressions:_col2 (type: string), _col3 (type: string), _col4 (type: string), _col5 (type: string)
- | | Select Operator [SEL_31]
- | | outputColumnNames:["_col0","_col2","_col3","_col4","_col5"]
- | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
- | | Filter Operator [FIL_105]
- | | predicate:((((((v1 = 'srv1') and k1 is not null) and k2 is not null) and k3 is not null) and v2 is not null) and v3 is not null) (type: boolean)
- | | Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
- | | TableScan [TS_29]
- | | alias:sr
- | | Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 17 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_38]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_34]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_106]
- | predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_32]
- | alias:d1
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 8 [SIMPLE_EDGE]
- Reduce Output Operator [RS_47]
- key expressions:_col8 (type: string), _col10 (type: string)
- Map-reduce partition columns:_col8 (type: string), _col10 (type: string)
- sort order:++
- Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
- value expressions:_col6 (type: string), _col7 (type: string)
- Merge Join Operator [MERGEJOIN_110]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col5 (type: string)","0":"_col1 (type: string)"}
- | outputColumnNames:["_col6","_col7","_col8","_col10"]
- | Statistics:Num rows: 665 Data size: 7069 Basic stats: COMPLETE Column stats: NONE
- |<-Map 7 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_42]
- | key expressions:_col1 (type: string)
- | Map-reduce partition columns:_col1 (type: string)
- | sort order:+
- | Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_7]
- | outputColumnNames:["_col1"]
- | Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_101]
- | predicate:((key = 'src1key') and value is not null) (type: boolean)
- | Statistics:Num rows: 6 Data size: 45 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_5]
- | alias:src1
- | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 11 [SIMPLE_EDGE]
- Reduce Output Operator [RS_44]
- key expressions:_col5 (type: string)
- Map-reduce partition columns:_col5 (type: string)
- sort order:+
- Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
- value expressions:_col4 (type: string), _col6 (type: string), _col8 (type: string)
- Merge Join Operator [MERGEJOIN_109]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col2 (type: string)","0":"_col0 (type: string)"}
- | outputColumnNames:["_col4","_col5","_col6","_col8"]
- | Statistics:Num rows: 605 Data size: 6427 Basic stats: COMPLETE Column stats: NONE
- |<-Map 10 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_24]
- | key expressions:_col0 (type: string)
- | Map-reduce partition columns:_col0 (type: string)
- | sort order:+
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_10]
- | outputColumnNames:["_col0"]
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_102]
- | predicate:((value = 'd1value') and key is not null) (type: boolean)
- | Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_8]
- | alias:d1
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 13 [SIMPLE_EDGE]
- Reduce Output Operator [RS_26]
- key expressions:_col2 (type: string)
- Map-reduce partition columns:_col2 (type: string)
- sort order:+
- Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
- value expressions:_col3 (type: string), _col4 (type: string), _col6 (type: string)
- Merge Join Operator [MERGEJOIN_108]
- | condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col3 (type: string)","0":"_col1 (type: string)"}
- | outputColumnNames:["_col2","_col3","_col4","_col6"]
- | Statistics:Num rows: 550 Data size: 5843 Basic stats: COMPLETE Column stats: NONE
- |<-Map 12 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_18]
- | key expressions:_col1 (type: string)
- | Map-reduce partition columns:_col1 (type: string)
- | sort order:+
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_13]
- | outputColumnNames:["_col1"]
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_103]
- | predicate:((key = 'srcpartkey') and value is not null) (type: boolean)
- | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_11]
- | alias:srcpart
- | Statistics:Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
- |<-Map 14 [SIMPLE_EDGE]
- Reduce Output Operator [RS_20]
- key expressions:_col3 (type: string)
- Map-reduce partition columns:_col3 (type: string)
- sort order:+
- Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
- value expressions:_col0 (type: string), _col1 (type: string), _col2 (type: string), _col4 (type: string)
- Select Operator [SEL_16]
- outputColumnNames:["_col0","_col1","_col2","_col3","_col4"]
- Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
- Filter Operator [FIL_104]
- predicate:((((((v3 = 'ssv3') and v2 is not null) and k1 is not null) and v1 is not null) and k2 is not null) and k3 is not null) (type: boolean)
- Statistics:Num rows: 2 Data size: 69 Basic stats: COMPLETE Column stats: NONE
- TableScan [TS_14]
- alias:ss
- Statistics:Num rows: 85 Data size: 2945 Basic stats: COMPLETE Column stats: NONE
+ Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+ value expressions:_col2 (type: string)
+ Merge Join Operator [MERGEJOIN_105]
+ | condition map:[{"":"Inner Join 0 to 1"}]
+ | keys:{"1":"_col0 (type: string)","0":"_col0 (type: string)"}
+ | outputColumnNames:["_col1","_col2","_col3"]
+ | Statistics:Num rows: 137 Data size: 1460 Basic stats: COMPLETE Column stats: NONE
+ |<-Map 1 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_51]
+ | key expressions:_col0 (type: string)
+ | Map-reduce partition columns:_col0 (type: string)
+ | sort order:+
+ | Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+ | value expressions:_col1 (type: string), _col2 (type: string), _col3 (type: string)
+ | Select Operator [SEL_1]
+ | outputColumnNames:["_col0","_col1","_col2","_col3"]
+ | Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+ | Filter Operator [FIL_97]
+ | predicate:((k1 is not null and v2 is not null) and v3 is not null) (type: boolean)
+ | Statistics:Num rows: 22 Data size: 762 Basic stats: COMPLETE Column stats: NONE
+ | TableScan [TS_0]
+ | alias:cs
+ | Statistics:Num rows: 170 Data size: 5890 Basic stats: COMPLETE Column stats: NONE
+ |<-Map 6 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_53]
+ key expressions:_col0 (type: string)
+ Map-reduce partition columns:_col0 (type: string)
+ sort order:+
+ Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ Select Operator [SEL_4]
+ outputColumnNames:["_col0"]
+ Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ Filter Operator [FIL_98]
+ predicate:((value) IN ('2000Q1', '2000Q2', '2000Q3') and key is not null) (type: boolean)
+ Statistics:Num rows: 125 Data size: 1328 Basic stats: COMPLETE Column stats: NONE
+ TableScan [TS_2]
+ alias:d1
+ Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
PREHOOK: query: explain
SELECT x.key, z.value, y.value
@@ -596,256 +596,256 @@ POSTHOOK: type: QUERY
Plan optimized by CBO.
Vertex dependency in root stage
-Map 13 <- Union 14 (CONTAINS)
-Map 17 <- Union 14 (CONTAINS)
-Map 5 <- Union 6 (CONTAINS)
-Map 9 <- Union 6 (CONTAINS)
-Reducer 12 <- Map 11 (SIMPLE_EDGE), Reducer 16 (SIMPLE_EDGE), Union 3 (CONTAINS)
-Reducer 15 <- Union 14 (SIMPLE_EDGE)
-Reducer 16 <- Map 18 (SIMPLE_EDGE), Reducer 15 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 8 (SIMPLE_EDGE), Union 3 (CONTAINS)
-Reducer 4 <- Union 3 (SIMPLE_EDGE)
+Map 1 <- Union 2 (CONTAINS)
+Map 11 <- Union 12 (CONTAINS)
+Map 16 <- Union 12 (CONTAINS)
+Map 8 <- Union 2 (CONTAINS)
+Reducer 13 <- Union 12 (SIMPLE_EDGE)
+Reducer 14 <- Map 17 (SIMPLE_EDGE), Reducer 13 (SIMPLE_EDGE)
+Reducer 15 <- Map 18 (SIMPLE_EDGE), Reducer 14 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 3 <- Union 2 (SIMPLE_EDGE)
+Reducer 4 <- Map 9 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 10 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
Reducer 7 <- Union 6 (SIMPLE_EDGE)
-Reducer 8 <- Map 10 (SIMPLE_EDGE), Reducer 7 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
- Reducer 4
- File Output Operator [FS_61]
+ Reducer 7
+ File Output Operator [FS_59]
compressed:false
Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
- Group By Operator [GBY_59]
+ Group By Operator [GBY_57]
| keys:KEY._col0 (type: string), KEY._col1 (type: string)
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- |<-Union 3 [SIMPLE_EDGE]
- |<-Reducer 2 [CONTAINS]
- | Reduce Output Operator [RS_58]
+ |<-Union 6 [SIMPLE_EDGE]
+ |<-Reducer 5 [CONTAINS]
+ | Reduce Output Operator [RS_56]
| key expressions:_col0 (type: string), _col1 (type: string)
| Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
| sort order:++
| Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
- | Group By Operator [GBY_57]
+ | Group By Operator [GBY_55]
| keys:_col0 (type: string), _col1 (type: string)
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_26]
+ | Select Operator [SEL_25]
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- | Merge Join Operator [MERGEJOIN_85]
+ | Merge Join Operator [MERGEJOIN_83]
| | condition map:[{"":"Inner Join 0 to 1"}]
- | | keys:{"1":"_col2 (type: string)","0":"_col0 (type: string)"}
- | | outputColumnNames:["_col2","_col3"]
+ | | keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+ | | outputColumnNames:["_col1","_col2"]
| | Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 1 [SIMPLE_EDGE]
- | | Reduce Output Operator [RS_22]
+ | |<-Map 10 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_23]
| | key expressions:_col0 (type: string)
| | Map-reduce partition columns:_col0 (type: string)
| | sort order:+
| | Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | | Select Operator [SEL_1]
+ | | Select Operator [SEL_14]
| | outputColumnNames:["_col0"]
| | Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | | Filter Operator [FIL_76]
+ | | Filter Operator [FIL_77]
| | predicate:key is not null (type: boolean)
| | Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | | TableScan [TS_0]
+ | | TableScan [TS_13]
| | alias:y
| | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- | |<-Reducer 8 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_24]
+ | |<-Reducer 4 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_21]
| key expressions:_col2 (type: string)
| Map-reduce partition columns:_col2 (type: string)
| sort order:+
| Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
| value expressions:_col1 (type: string)
- | Merge Join Operator [MERGEJOIN_84]
+ | Merge Join Operator [MERGEJOIN_82]
| | condition map:[{"":"Inner Join 0 to 1"}]
| | keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
| | outputColumnNames:["_col1","_col2"]
| | Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 10 [SIMPLE_EDGE]
+ | |<-Map 9 [SIMPLE_EDGE]
| | Reduce Output Operator [RS_18]
| | key expressions:_col1 (type: string)
| | Map-reduce partition columns:_col1 (type: string)
| | sort order:+
| | Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
| | value expressions:_col0 (type: string)
- | | Select Operator [SEL_14]
+ | | Select Operator [SEL_12]
| | outputColumnNames:["_col0","_col1"]
| | Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- | | Filter Operator [FIL_79]
+ | | Filter Operator [FIL_76]
| | predicate:(value is not null and key is not null) (type: boolean)
| | Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- | | TableScan [TS_13]
+ | | TableScan [TS_11]
| | alias:x
| | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
- | |<-Reducer 7 [SIMPLE_EDGE]
+ | |<-Reducer 3 [SIMPLE_EDGE]
| Reduce Output Operator [RS_16]
| key expressions:_col1 (type: string)
| Map-reduce partition columns:_col1 (type: string)
| sort order:+
| Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_12]
+ | Select Operator [SEL_10]
| outputColumnNames:["_col1"]
| Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
- | Group By Operator [GBY_11]
+ | Group By Operator [GBY_9]
| | keys:KEY._col0 (type: string), KEY._col1 (type: string)
| | outputColumnNames:["_col0","_col1"]
| | Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
- | |<-Union 6 [SIMPLE_EDGE]
- | |<-Map 5 [CONTAINS]
- | | Reduce Output Operator [RS_10]
+ | |<-Union 2 [SIMPLE_EDGE]
+ | |<-Map 1 [CONTAINS]
+ | | Reduce Output Operator [RS_8]
| | key expressions:_col0 (type: string), _col1 (type: string)
| | Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
| | sort order:++
| | Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- | | Group By Operator [GBY_9]
+ | | Group By Operator [GBY_7]
| | keys:_col0 (type: string), _col1 (type: string)
| | outputColumnNames:["_col0","_col1"]
| | Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- | | Select Operator [SEL_3]
+ | | Select Operator [SEL_1]
| | outputColumnNames:["_col0","_col1"]
| | Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
- | | Filter Operator [FIL_77]
+ | | Filter Operator [FIL_74]
| | predicate:value is not null (type: boolean)
| | Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
- | | TableScan [TS_2]
+ | | TableScan [TS_0]
| | alias:x
| | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 9 [CONTAINS]
- | Reduce Output Operator [RS_10]
+ | |<-Map 8 [CONTAINS]
+ | Reduce Output Operator [RS_8]
| key expressions:_col0 (type: string), _col1 (type: string)
| Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
| sort order:++
| Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- | Group By Operator [GBY_9]
+ | Group By Operator [GBY_7]
| keys:_col0 (type: string), _col1 (type: string)
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_5]
+ | Select Operator [SEL_3]
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_78]
+ | Filter Operator [FIL_75]
| predicate:value is not null (type: boolean)
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_4]
+ | TableScan [TS_2]
| alias:y
| Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 12 [CONTAINS]
- Reduce Output Operator [RS_58]
+ |<-Reducer 15 [CONTAINS]
+ Reduce Output Operator [RS_56]
key expressions:_col0 (type: string), _col1 (type: string)
Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
sort order:++
Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
- Group By Operator [GBY_57]
+ Group By Operator [GBY_55]
keys:_col0 (type: string), _col1 (type: string)
outputColumnNames:["_col0","_col1"]
Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
- Select Operator [SEL_53]
+ Select Operator [SEL_51]
outputColumnNames:["_col0","_col1"]
Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- Merge Join Operator [MERGEJOIN_87]
+ Merge Join Operator [MERGEJOIN_85]
| condition map:[{"":"Inner Join 0 to 1"}]
- | keys:{"1":"_col2 (type: string)","0":"_col0 (type: string)"}
- | outputColumnNames:["_col2","_col3"]
+ | keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+ | outputColumnNames:["_col1","_col2"]
| Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- |<-Map 11 [SIMPLE_EDGE]
+ |<-Map 18 [SIMPLE_EDGE]
| Reduce Output Operator [RS_49]
| key expressions:_col0 (type: string)
| Map-reduce partition columns:_col0 (type: string)
| sort order:+
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_28]
+ | Select Operator [SEL_40]
| outputColumnNames:["_col0"]
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_80]
+ | Filter Operator [FIL_81]
| predicate:key is not null (type: boolean)
| Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_27]
+ | TableScan [TS_39]
| alias:y
| Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 16 [SIMPLE_EDGE]
- Reduce Output Operator [RS_51]
+ |<-Reducer 14 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_47]
key expressions:_col2 (type: string)
Map-reduce partition columns:_col2 (type: string)
sort order:+
Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
value expressions:_col1 (type: string)
- Merge Join Operator [MERGEJOIN_86]
+ Merge Join Operator [MERGEJOIN_84]
| condition map:[{"":"Inner Join 0 to 1"}]
| keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
| outputColumnNames:["_col1","_col2"]
| Statistics:Num rows: 144 Data size: 1509 Basic stats: COMPLETE Column stats: NONE
- |<-Map 18 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_45]
+ |<-Map 17 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_44]
| key expressions:_col1 (type: string)
| Map-reduce partition columns:_col1 (type: string)
| sort order:+
| Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
| value expressions:_col0 (type: string)
- | Select Operator [SEL_41]
+ | Select Operator [SEL_38]
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_83]
+ | Filter Operator [FIL_80]
| predicate:(value is not null and key is not null) (type: boolean)
| Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_40]
+ | TableScan [TS_37]
| alias:x
| Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
- |<-Reducer 15 [SIMPLE_EDGE]
- Reduce Output Operator [RS_43]
+ |<-Reducer 13 [SIMPLE_EDGE]
+ Reduce Output Operator [RS_42]
key expressions:_col1 (type: string)
Map-reduce partition columns:_col1 (type: string)
sort order:+
Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
- Select Operator [SEL_39]
+ Select Operator [SEL_36]
outputColumnNames:["_col1"]
Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
- Group By Operator [GBY_38]
+ Group By Operator [GBY_35]
| keys:KEY._col0 (type: string), KEY._col1 (type: string)
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 131 Data size: 1372 Basic stats: COMPLETE Column stats: NONE
- |<-Union 14 [SIMPLE_EDGE]
- |<-Map 13 [CONTAINS]
- | Reduce Output Operator [RS_37]
+ |<-Union 12 [SIMPLE_EDGE]
+ |<-Map 11 [CONTAINS]
+ | Reduce Output Operator [RS_34]
| key expressions:_col0 (type: string), _col1 (type: string)
| Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
| sort order:++
| Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- | Group By Operator [GBY_36]
+ | Group By Operator [GBY_33]
| keys:_col0 (type: string), _col1 (type: string)
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_30]
+ | Select Operator [SEL_27]
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
- | Filter Operator [FIL_81]
+ | Filter Operator [FIL_78]
| predicate:value is not null (type: boolean)
| Statistics:Num rows: 13 Data size: 99 Basic stats: COMPLETE Column stats: NONE
- | TableScan [TS_29]
+ | TableScan [TS_26]
| alias:x
| Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
- |<-Map 17 [CONTAINS]
- Reduce Output Operator [RS_37]
+ |<-Map 16 [CONTAINS]
+ Reduce Output Operator [RS_34]
key expressions:_col0 (type: string), _col1 (type: string)
Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
sort order:++
Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- Group By Operator [GBY_36]
+ Group By Operator [GBY_33]
keys:_col0 (type: string), _col1 (type: string)
outputColumnNames:["_col0","_col1"]
Statistics:Num rows: 263 Data size: 2755 Basic stats: COMPLETE Column stats: NONE
- Select Operator [SEL_32]
+ Select Operator [SEL_29]
outputColumnNames:["_col0","_col1"]
Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- Filter Operator [FIL_82]
+ Filter Operator [FIL_79]
predicate:value is not null (type: boolean)
Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- TableScan [TS_31]
+ TableScan [TS_28]
alias:y
Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
@@ -878,500 +878,500 @@ POSTHOOK: type: QUERY
Plan optimized by CBO.
Vertex dependency in root stage
-Map 11 <- Union 8 (CONTAINS)
-Map 15 <- Union 16 (CONTAINS)
+Map 1 <- Union 2 (CONTAINS)
+Map 10 <- Union 2 (CONTAINS)
+Map 13 <- Union 14 (CONTAINS)
+Map 20 <- Union 14 (CONTAINS)
Map 21 <- Union 16 (CONTAINS)
-Map 22 <- Union 18 (CONTAINS)
-Map 26 <- Union 27 (CONTAINS)
+Map 24 <- Union 25 (CONTAINS)
+Map 33 <- Union 25 (CONTAINS)
Map 34 <- Union 27 (CONTAINS)
Map 35 <- Union 29 (CONTAINS)
-Map 36 <- Union 31 (CONTAINS)
-Map 7 <- Union 8 (CONTAINS)
-Reducer 10 <- Map 12 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE)
-Reducer 14 <- Map 13 (SIMPLE_EDGE), Reducer 20 (SIMPLE_EDGE), Union 3 (CONTAINS)
-Reducer 17 <- Union 16 (SIMPLE_EDGE), Union 18 (CONTAINS)
-Reducer 19 <- Union 18 (SIMPLE_EDGE)
-Reducer 2 <- Map 1 (SIMPLE_EDGE), Reducer 10 (SIMPLE_EDGE), Union 3 (CONTAINS)
-Reducer 20 <- Map 23 (SIMPLE_EDGE), Reducer 19 (SIMPLE_EDGE)
-Reducer 25 <- Map 24 (SIMPLE_EDGE), Reducer 33 (SIMPLE_EDGE), Union 5 (CONTAINS)
+Reducer 15 <- Union 14 (SIMPLE_EDGE), Union 16 (CONTAINS)
+Reducer 17 <- Union 16 (SIMPLE_EDGE)
+Reducer 18 <- Map 22 (SIMPLE_EDGE), Reducer 17 (SIMPLE_EDGE)
+Reducer 19 <- Map 23 (SIMPLE_EDGE), Reducer 18 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 26 <- Union 25 (SIMPLE_EDGE), Union 27 (CONTAINS)
Reducer 28 <- Union 27 (SIMPLE_EDGE), Union 29 (CONTAINS)
-Reducer 30 <- Union 29 (SIMPLE_EDGE), Union 31 (CONTAINS)
-Reducer 32 <- Union 31 (SIMPLE_EDGE)
-Reducer 33 <- Map 37 (SIMPLE_EDGE), Reducer 32 (SIMPLE_EDGE)
-Reducer 4 <- Union 3 (SIMPLE_EDGE), Union 5 (CONTAINS)
-Reducer 6 <- Union 5 (SIMPLE_EDGE)
+Reducer 3 <- Union 2 (SIMPLE_EDGE)
+Reducer 30 <- Union 29 (SIMPLE_EDGE)
+Reducer 31 <- Map 36 (SIMPLE_EDGE), Reducer 30 (SIMPLE_EDGE)
+Reducer 32 <- Map 37 (SIMPLE_EDGE), Reducer 31 (SIMPLE_EDGE), Union 8 (CONTAINS)
+Reducer 4 <- Map 11 (SIMPLE_EDGE), Reducer 3 (SIMPLE_EDGE)
+Reducer 5 <- Map 12 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE), Union 6 (CONTAINS)
+Reducer 7 <- Union 6 (SIMPLE_EDGE), Union 8 (CONTAINS)
Reducer 9 <- Union 8 (SIMPLE_EDGE)
Stage-0
Fetch Operator
limit:-1
Stage-1
- Reducer 6
- File Output Operator [FS_122]
+ Reducer 9
+ File Output Operator [FS_119]
compressed:false
Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
table:{"serde:":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","input format:":"org.apache.hadoop.mapred.TextInputFormat","output format:":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat"}
- Group By Operator [GBY_120]
+ Group By Operator [GBY_117]
| keys:KEY._col0 (type: string), KEY._col1 (type: string)
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- |<-Union 5 [SIMPLE_EDGE]
- |<-Reducer 25 [CONTAINS]
- | Reduce Output Operator [RS_119]
+ |<-Union 8 [SIMPLE_EDGE]
+ |<-Reducer 32 [CONTAINS]
+ | Reduce Output Operator [RS_116]
| key expressions:_col0 (type: string), _col1 (type: string)
| Map-reduce partition columns:_col0 (type: string), _col1 (type: string)
| sort order:++
| Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
- | Group By Operator [GBY_118]
+ | Group By Operator [GBY_115]
| keys:_col0 (type: string), _col1 (type: string)
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 550 Data size: 5842 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_114]
+ | Select Operator [SEL_111]
| outputColumnNames:["_col0","_col1"]
| Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- | Merge Join Operator [MERGEJOIN_170]
+ | Merge Join Operator [MERGEJOIN_167]
| | condition map:[{"":"Inner Join 0 to 1"}]
- | | keys:{"1":"_col2 (type: string)","0":"_col0 (type: string)"}
- | | outputColumnNames:["_col1","_col4"]
+ | | keys:{"1":"_col0 (type: string)","0":"_col2 (type: string)"}
+ | | outputColumnNames:["_col2","_col5"]
| | Statistics:Num rows: 275 Data size: 2921 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 24 [SIMPLE_EDGE]
- | | Reduce Output Operator [RS_110]
+ | |<-Map 37 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_109]
| | key expressions:_col0 (type: string)
| | Map-reduce partition columns:_col0 (type: string)
| | sort order:+
| | Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
| | value expressions:_col1 (type: string)
- | | Select Operator [SEL_71]
+ | | Select Operator [SEL_100]
| | outputColumnNames:["_col0","_col1"]
| | Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | | Filter Operator [FIL_159]
+ | | Filter Operator [FIL_161]
| | predicate:key is not null (type: boolean)
| | Statistics:Num rows: 250 Data size: 2656 Basic stats: COMPLETE Column stats: NONE
- | | TableScan [TS_70]
+ | | TableScan [TS_99]
| | alias:y
| | Statistics:Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
- | |<-Reducer 33 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_112]
+ | |<-Reducer 31 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_107]
| key expressions:_col2 (type: string)
| Map-reduce partition columns:_col2 (type: string)
| sort order:+
| Statistics:Num rows: 242 Data size: 2565 Basic stats: COMPLETE Column stats: NONE
- | Merge Join Operator [MERGEJOIN_169]
+ | Merge Join Operator [MERGEJOIN_166]
| | condition map:[{"":"Inner Join 0 to 1"}]
| | keys:{"1":"_col1 (type: string)","0":"_col1 (type: string)"}
| | outputColumnNames:["_col2"]
| | Statistics:Num rows: 242 Data size: 2565 Basic stats: COMPLETE Column stats: NONE
- | |<-Map 37 [SIMPLE_EDGE]
- | | Reduce Output Operator [RS_106]
+ | |<-Map 36 [SIMPLE_EDGE]
+ | | Reduce Output Operator [RS_104]
| | key expressions:_col1 (type: string)
| | Map-reduce partition columns:_col1 (type: string)
| | sort order:+
| | Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
| | value expressions:_col0 (type: string)
- | | Select Operator [SEL_102]
+ | | Select Operator [SEL_98]
| | outputColumnNames:["_col0","_col1"]
| | Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- | | Filter Operator [FIL_164]
+ | | Filter Operator [FIL_160]
| | predicate:(value is not null and key is not null) (type: boolean)
| | Statistics:Num rows: 7 Data size: 53 Basic stats: COMPLETE Column stats: NONE
- | | TableScan [TS_101]
+ | | TableScan [TS_97]
| | alias:x
| | Statistics:Num rows: 25 Data size: 191 Basic stats: COMPLETE Column stats: NONE
- | |<-Reducer 32 [SIMPLE_EDGE]
- | Reduce Output Operator [RS_104]
+ | |<-Reducer 30 [SIMPLE_EDGE]
+ | Reduce Output Operator [RS_102]
| key expressions:_col1 (type: string)
| Map-reduce partition columns:_col1 (type: string)
| sort order:+
| Statistics:Num rows: 220 Data size: 2332 Basic stats: COMPLETE Column stats: NONE
- | Select Operator [SEL_100]
+ | Select Operator [SEL_96]
| outputColumnNames:["_col1"]
| Statistics:Num rows: 220 Data size: 2332 Basic stat
<TRUNCATED>
[07/25] hive git commit: HIVE-10165 Improve hive-hcatalog-streaming
extensibility and support updates and deletes (Eliot West via gates)
Posted by se...@apache.org.
HIVE-10165 Improve hive-hcatalog-streaming extensibility and support updates and deletes (Eliot West via gates)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/994d98c0
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/994d98c0
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/994d98c0
Branch: refs/heads/llap
Commit: 994d98c0963ee48c2abbfee6f389d75c0223c8f1
Parents: 3991dba
Author: Alan Gates <ga...@hortonworks.com>
Authored: Tue Jun 30 14:59:55 2015 -0700
Committer: Alan Gates <ga...@hortonworks.com>
Committed: Tue Jun 30 14:59:55 2015 -0700
----------------------------------------------------------------------
.gitignore | 1 +
hcatalog/streaming/pom.xml | 6 +
.../streaming/mutate/HiveConfFactory.java | 63 +++
.../mutate/UgiMetaStoreClientFactory.java | 102 ++++
.../streaming/mutate/client/AcidTable.java | 112 ++++
.../mutate/client/AcidTableSerializer.java | 100 ++++
.../mutate/client/ClientException.java | 15 +
.../mutate/client/ConnectionException.java | 15 +
.../streaming/mutate/client/MutatorClient.java | 140 +++++
.../mutate/client/MutatorClientBuilder.java | 115 ++++
.../streaming/mutate/client/TableType.java | 37 ++
.../streaming/mutate/client/Transaction.java | 114 ++++
.../mutate/client/TransactionException.java | 15 +
.../mutate/client/lock/HeartbeatFactory.java | 30 +
.../mutate/client/lock/HeartbeatTimerTask.java | 66 +++
.../streaming/mutate/client/lock/Lock.java | 282 ++++++++++
.../mutate/client/lock/LockException.java | 15 +
.../mutate/client/lock/LockFailureListener.java | 26 +
.../mutate/doc-files/system-overview.dot | 27 +
.../hive/hcatalog/streaming/mutate/package.html | 495 +++++++++++++++++
.../mutate/worker/BucketIdException.java | 11 +
.../mutate/worker/BucketIdResolver.java | 11 +
.../mutate/worker/BucketIdResolverImpl.java | 76 +++
.../mutate/worker/CreatePartitionHelper.java | 83 +++
.../mutate/worker/GroupRevisitedException.java | 11 +
.../mutate/worker/GroupingValidator.java | 74 +++
.../streaming/mutate/worker/Mutator.java | 21 +
.../mutate/worker/MutatorCoordinator.java | 281 ++++++++++
.../worker/MutatorCoordinatorBuilder.java | 76 +++
.../streaming/mutate/worker/MutatorFactory.java | 16 +
.../streaming/mutate/worker/MutatorImpl.java | 84 +++
.../streaming/mutate/worker/OperationType.java | 7 +
.../worker/PartitionCreationException.java | 15 +
.../mutate/worker/RecordInspector.java | 11 +
.../mutate/worker/RecordInspectorImpl.java | 45 ++
.../mutate/worker/RecordSequenceException.java | 11 +
.../mutate/worker/SequenceValidator.java | 49 ++
.../mutate/worker/WorkerException.java | 15 +
.../streaming/mutate/ExampleUseCase.java | 82 +++
.../streaming/mutate/MutableRecord.java | 50 ++
.../mutate/ReflectiveMutatorFactory.java | 51 ++
.../streaming/mutate/StreamingAssert.java | 191 +++++++
.../streaming/mutate/StreamingTestUtils.java | 261 +++++++++
.../streaming/mutate/TestMutations.java | 544 +++++++++++++++++++
.../mutate/client/TestAcidTableSerializer.java | 66 +++
.../mutate/client/TestMutatorClient.java | 176 ++++++
.../mutate/client/TestTransaction.java | 95 ++++
.../client/lock/TestHeartbeatTimerTask.java | 100 ++++
.../streaming/mutate/client/lock/TestLock.java | 283 ++++++++++
.../mutate/worker/TestBucketIdResolverImpl.java | 38 ++
.../mutate/worker/TestGroupingValidator.java | 70 +++
.../mutate/worker/TestMutatorCoordinator.java | 234 ++++++++
.../mutate/worker/TestMutatorImpl.java | 99 ++++
.../mutate/worker/TestRecordInspectorImpl.java | 31 ++
.../mutate/worker/TestSequenceValidator.java | 91 ++++
55 files changed, 5135 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index c5decaf..4d341a0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,3 +27,4 @@ hcatalog/webhcat/java-client/target
hcatalog/storage-handlers/hbase/target
hcatalog/webhcat/svr/target
conf/hive-default.xml.template
+.DS_Store
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/pom.xml b/hcatalog/streaming/pom.xml
index 2135e89..6d03ce1 100644
--- a/hcatalog/streaming/pom.xml
+++ b/hcatalog/streaming/pom.xml
@@ -89,6 +89,12 @@
<optional>true</optional>
<version>${project.version}</version>
</dependency>
+ <dependency>
+ <groupId>org.apache.commons</groupId>
+ <artifactId>commons-lang3</artifactId>
+ <optional>true</optional>
+ <version>3.3.2</version>
+ </dependency>
<!-- test -->
<dependency>
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java
new file mode 100644
index 0000000..fcf446c
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/HiveConfFactory.java
@@ -0,0 +1,63 @@
+package org.apache.hive.hcatalog.streaming.mutate;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Creates/configures {@link HiveConf} instances with required ACID attributes. */
+public class HiveConfFactory {
+
+ private static final Logger LOG = LoggerFactory.getLogger(HiveConfFactory.class);
+ private static final String TRANSACTION_MANAGER = "org.apache.hadoop.hive.ql.lockmgr.DbTxnManager";
+
+ public static HiveConf newInstance(Configuration configuration, Class<?> clazz, String metaStoreUri) {
+ HiveConf hiveConf = null;
+ if (configuration != null) {
+ if (!HiveConf.class.isAssignableFrom(configuration.getClass())) {
+ hiveConf = new HiveConf(configuration, clazz);
+ } else {
+ hiveConf = (HiveConf) configuration;
+ }
+ }
+
+ if (hiveConf == null) {
+ hiveConf = HiveConfFactory.newInstance(clazz, metaStoreUri);
+ } else {
+ HiveConfFactory.overrideSettings(hiveConf);
+ }
+ return hiveConf;
+ }
+
+ public static HiveConf newInstance(Class<?> clazz, String metaStoreUri) {
+ HiveConf conf = new HiveConf(clazz);
+ if (metaStoreUri != null) {
+ setHiveConf(conf, HiveConf.ConfVars.METASTOREURIS, metaStoreUri);
+ }
+ overrideSettings(conf);
+ return conf;
+ }
+
+ public static void overrideSettings(HiveConf conf) {
+ setHiveConf(conf, HiveConf.ConfVars.HIVE_TXN_MANAGER, TRANSACTION_MANAGER);
+ setHiveConf(conf, HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+ setHiveConf(conf, HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true);
+ // Avoids creating Tez Client sessions internally as it takes much longer currently
+ setHiveConf(conf, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE, "mr");
+ }
+
+ private static void setHiveConf(HiveConf conf, HiveConf.ConfVars var, String value) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Overriding HiveConf setting : {} = {}", var, value);
+ }
+ conf.setVar(var, value);
+ }
+
+ private static void setHiveConf(HiveConf conf, HiveConf.ConfVars var, boolean value) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Overriding HiveConf setting : {} = {}", var, value);
+ }
+ conf.setBoolVar(var, value);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java
new file mode 100644
index 0000000..2a4ddbe
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/UgiMetaStoreClientFactory.java
@@ -0,0 +1,102 @@
+package org.apache.hive.hcatalog.streaming.mutate;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.thrift.TException;
+
+import com.google.common.reflect.AbstractInvocationHandler;
+
+/**
+ * Creates a proxied {@link IMetaStoreClient client} that wraps calls in a {@link PrivilegedExceptionAction} if the
+ * {@link UserGroupInformation} is specified. Invokes directly otherwise.
+ */
+public class UgiMetaStoreClientFactory {
+
+ private static Set<Method> I_META_STORE_CLIENT_METHODS = getIMetaStoreClientMethods();
+
+ private final String metaStoreUri;
+ private final HiveConf conf;
+ private final boolean secureMode;
+ private final UserGroupInformation authenticatedUser;
+ private final String user;
+
+ public UgiMetaStoreClientFactory(String metaStoreUri, HiveConf conf, UserGroupInformation authenticatedUser,
+ String user, boolean secureMode) {
+ this.metaStoreUri = metaStoreUri;
+ this.conf = conf;
+ this.authenticatedUser = authenticatedUser;
+ this.user = user;
+ this.secureMode = secureMode;
+ if (metaStoreUri != null) {
+ conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreUri);
+ }
+ if (secureMode) {
+ conf.setBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL, true);
+ }
+ }
+
+ public IMetaStoreClient newInstance() throws MetaException {
+ return newInstance(new HiveMetaStoreClient(conf));
+ }
+
+ public IMetaStoreClient newInstance(IMetaStoreClient delegate) throws MetaException {
+ return createProxy(delegate, user, authenticatedUser);
+ }
+
+ @Override
+ public String toString() {
+ return "UgiMetaStoreClientFactory [metaStoreUri=" + metaStoreUri + ", secureMode=" + secureMode
+ + ", authenticatedUser=" + authenticatedUser + ", user=" + user + "]";
+ }
+
+ private IMetaStoreClient createProxy(final IMetaStoreClient delegate, final String user,
+ final UserGroupInformation authenticatedUser) {
+ InvocationHandler handler = new AbstractInvocationHandler() {
+
+ @Override
+ protected Object handleInvocation(Object proxy, final Method method, final Object[] args) throws Throwable {
+ try {
+ if (!I_META_STORE_CLIENT_METHODS.contains(method) || authenticatedUser == null) {
+ return method.invoke(delegate, args);
+ }
+ try {
+ return authenticatedUser.doAs(new PrivilegedExceptionAction<Object>() {
+ @Override
+ public Object run() throws Exception {
+ return method.invoke(delegate, args);
+ }
+ });
+ } catch (IOException | InterruptedException e) {
+ throw new TException("PrivilegedExceptionAction failed as user '" + user + "'.", e);
+ }
+ } catch (UndeclaredThrowableException | InvocationTargetException e) {
+ throw e.getCause();
+ }
+ }
+ };
+
+ ClassLoader classLoader = IMetaStoreClient.class.getClassLoader();
+ Class<?>[] interfaces = new Class<?>[] { IMetaStoreClient.class };
+ Object proxy = Proxy.newProxyInstance(classLoader, interfaces, handler);
+ return IMetaStoreClient.class.cast(proxy);
+ }
+
+ private static Set<Method> getIMetaStoreClientMethods() {
+ return new HashSet<>(Arrays.asList(IMetaStoreClient.class.getDeclaredMethods()));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java
new file mode 100644
index 0000000..20747db
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTable.java
@@ -0,0 +1,112 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+/**
+ * Describes an ACID table that can receive mutation events. Used to encode the information required by workers to write
+ * ACID events without requiring them to once more retrieve the data from the meta store db.
+ */
+public class AcidTable implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String databaseName;
+ private final String tableName;
+ private final boolean createPartitions;
+ private final TableType tableType;
+ private long transactionId;
+
+ private Table table;
+
+ AcidTable(String databaseName, String tableName, boolean createPartitions, TableType tableType) {
+ this.databaseName = databaseName;
+ this.tableName = tableName;
+ this.createPartitions = createPartitions;
+ this.tableType = tableType;
+ }
+
+ /**
+ * Returns {@code 0} until such a time that a {@link Transaction} has been acquired (when
+ * {@link MutatorClient#newTransaction()} exits), at which point this will return the
+ * {@link Transaction#getTransactionId() transaction id}.
+ */
+ public long getTransactionId() {
+ return transactionId;
+ }
+
+ public String getDatabaseName() {
+ return databaseName;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ public boolean createPartitions() {
+ return createPartitions;
+ }
+
+ /**
+ * Returns {@code null} until such a time that the table described by the {@link #getDatabaseName() database_name}
+ * {@code .}{@link #getTableName() table_name} has been resolved with the meta store database (when
+ * {@link MutatorClient#connect()} exits), at which point this will then return the corresponding
+ * {@link StorageDescriptor#getOutputFormat() OutputFormat}.
+ */
+ public String getOutputFormatName() {
+ return table != null ? table.getSd().getOutputFormat() : null;
+ }
+
+ /**
+ * Returns {@code 0} until such a time that the table described by the {@link #getDatabaseName() database_name}
+ * {@code .}{@link #getTableName() table_name} has been resolved with the meta store database (when
+ * {@link MutatorClient#connect()} exits), at which point this will then return the corresponding
+ * {@link StorageDescriptor#getNumBuckets() total bucket count}.
+ */
+ public int getTotalBuckets() {
+ return table != null ? table.getSd().getNumBuckets() : 0;
+ }
+
+ public TableType getTableType() {
+ return tableType;
+ }
+
+ public String getQualifiedName() {
+ return (databaseName + "." + tableName).toUpperCase();
+ }
+
+ /**
+ * Returns {@code null} until such a time that the table described by the {@link #getDatabaseName() database_name}
+ * {@code .}{@link #getTableName() table_name} has been resolved with the meta store database (when
+ * {@link MutatorClient#connect()} exits), at which point this will then return the corresponding {@link Table}.
+ * Provided as a convenience to API users who may wish to gather further meta data regarding the table without
+ * connecting with the meta store once more.
+ */
+ public Table getTable() {
+ return table;
+ }
+
+ void setTransactionId(long transactionId) {
+ this.transactionId = transactionId;
+ }
+
+ void setTable(Table table) {
+ if (!databaseName.equalsIgnoreCase(table.getDbName())) {
+ throw new IllegalArgumentException("Incorrect database name.");
+ }
+ if (!tableName.equalsIgnoreCase(table.getTableName())) {
+ throw new IllegalArgumentException("Incorrect table name.");
+ }
+ this.table = table;
+ }
+
+ @Override
+ public String toString() {
+ return "AcidTable [databaseName=" + databaseName + ", tableName=" + tableName + ", createPartitions="
+ + createPartitions + ", tableType=" + tableType + ", outputFormatName=" + getOutputFormatName()
+ + ", totalBuckets=" + getTotalBuckets() + ", transactionId=" + transactionId + "]";
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java
new file mode 100644
index 0000000..5d8a2bf
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/AcidTableSerializer.java
@@ -0,0 +1,100 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.charset.Charset;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TDeserializer;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TCompactProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility to serialize/deserialize {@link AcidTable AcidTables} into strings so that they can be easily transported as
+ * {@link Configuration} properties.
+ */
+public class AcidTableSerializer {
+
+ private static final Logger LOG = LoggerFactory.getLogger(AcidTableSerializer.class);
+
+ /* Allow for improved schemes. */
+ private static final String PROLOG_V1 = "AcidTableV1:";
+
+ /** Returns a base 64 encoded representation of the supplied {@link AcidTable}. */
+ public static String encode(AcidTable table) throws IOException {
+ DataOutputStream data = null;
+ ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+ try {
+ data = new DataOutputStream(bytes);
+ data.writeUTF(table.getDatabaseName());
+ data.writeUTF(table.getTableName());
+ data.writeBoolean(table.createPartitions());
+ if (table.getTransactionId() <= 0) {
+ LOG.warn("Transaction ID <= 0. The recipient is probably expecting a transaction ID.");
+ }
+ data.writeLong(table.getTransactionId());
+ data.writeByte(table.getTableType().getId());
+
+ Table metaTable = table.getTable();
+ if (metaTable != null) {
+ byte[] thrift = new TSerializer(new TCompactProtocol.Factory()).serialize(metaTable);
+ data.writeInt(thrift.length);
+ data.write(thrift);
+ } else {
+ LOG.warn("Meta store table is null. The recipient is probably expecting an instance.");
+ data.writeInt(0);
+ }
+ } catch (TException e) {
+ throw new IOException("Error serializing meta store table.", e);
+ } finally {
+ data.close();
+ }
+
+ return PROLOG_V1 + new String(Base64.encodeBase64(bytes.toByteArray()), Charset.forName("UTF-8"));
+ }
+
+ /** Returns the {@link AcidTable} instance decoded from a base 64 representation. */
+ public static AcidTable decode(String encoded) throws IOException {
+ if (!encoded.startsWith(PROLOG_V1)) {
+ throw new IllegalStateException("Unsupported version.");
+ }
+ encoded = encoded.substring(PROLOG_V1.length());
+
+ byte[] decoded = Base64.decodeBase64(encoded);
+ AcidTable table = null;
+ try (DataInputStream in = new DataInputStream(new ByteArrayInputStream(decoded))) {
+ String databaseName = in.readUTF();
+ String tableName = in.readUTF();
+ boolean createPartitions = in.readBoolean();
+ long transactionId = in.readLong();
+ TableType tableType = TableType.valueOf(in.readByte());
+ int thriftLength = in.readInt();
+
+ table = new AcidTable(databaseName, tableName, createPartitions, tableType);
+ table.setTransactionId(transactionId);
+
+ Table metaTable = null;
+ if (thriftLength > 0) {
+ metaTable = new Table();
+ try {
+ byte[] thriftEncoded = new byte[thriftLength];
+ in.readFully(thriftEncoded, 0, thriftLength);
+ new TDeserializer(new TCompactProtocol.Factory()).deserialize(metaTable, thriftEncoded);
+ table.setTable(metaTable);
+ } catch (TException e) {
+ throw new IOException("Error deserializing meta store table.", e);
+ }
+ }
+ }
+ return table;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java
new file mode 100644
index 0000000..988dc38
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ClientException.java
@@ -0,0 +1,15 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+public class ClientException extends Exception {
+
+ private static final long serialVersionUID = 1L;
+
+ ClientException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ ClientException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java
new file mode 100644
index 0000000..b54455a
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/ConnectionException.java
@@ -0,0 +1,15 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+public class ConnectionException extends ClientException {
+
+ private static final long serialVersionUID = 1L;
+
+ ConnectionException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ ConnectionException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java
new file mode 100644
index 0000000..2724525
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClient.java
@@ -0,0 +1,140 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockFailureListener;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Responsible for orchestrating {@link Transaction Transactions} within which ACID table mutation events can occur.
+ * Typically this will be a large batch of delta operations.
+ */
+public class MutatorClient implements Closeable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(MutatorClient.class);
+ private static final String TRANSACTIONAL_PARAM_KEY = "transactional";
+
+ private final IMetaStoreClient metaStoreClient;
+ private final Lock.Options lockOptions;
+ private final List<AcidTable> tables;
+ private boolean connected;
+
+ MutatorClient(IMetaStoreClient metaStoreClient, HiveConf configuration, LockFailureListener lockFailureListener,
+ String user, Collection<AcidTable> tables) {
+ this.metaStoreClient = metaStoreClient;
+ this.tables = Collections.unmodifiableList(new ArrayList<>(tables));
+
+ lockOptions = new Lock.Options()
+ .configuration(configuration)
+ .lockFailureListener(lockFailureListener == null ? LockFailureListener.NULL_LISTENER : lockFailureListener)
+ .user(user);
+ for (AcidTable table : tables) {
+ lockOptions.addTable(table.getDatabaseName(), table.getTableName());
+ }
+ }
+
+ /**
+ * Connects to the {@link IMetaStoreClient meta store} that will be used to manage {@link Transaction} life-cycles.
+ * Also checks that the tables destined to receive mutation events are able to do so. The client should only hold one
+ * open transaction at any given time (TODO: enforce this).
+ */
+ public void connect() throws ConnectionException {
+ if (connected) {
+ throw new ConnectionException("Already connected.");
+ }
+ for (AcidTable table : tables) {
+ checkTable(metaStoreClient, table);
+ }
+ LOG.debug("Connected to end point {}", metaStoreClient);
+ connected = true;
+ }
+
+ /** Creates a new {@link Transaction} by opening a transaction with the {@link IMetaStoreClient meta store}. */
+ public Transaction newTransaction() throws TransactionException {
+ if (!connected) {
+ throw new TransactionException("Not connected - cannot create transaction.");
+ }
+ Transaction transaction = new Transaction(metaStoreClient, lockOptions);
+ for (AcidTable table : tables) {
+ table.setTransactionId(transaction.getTransactionId());
+ }
+ LOG.debug("Created transaction {}", transaction);
+ return transaction;
+ }
+
+ /** Did the client connect successfully. Note the the client may have since become disconnected. */
+ public boolean isConnected() {
+ return connected;
+ }
+
+ /**
+ * Closes the client releasing any {@link IMetaStoreClient meta store} connections held. Does not notify any open
+ * transactions (TODO: perhaps it should?)
+ */
+ @Override
+ public void close() throws IOException {
+ metaStoreClient.close();
+ LOG.debug("Closed client.");
+ connected = false;
+ }
+
+ /**
+ * Returns the list of managed {@link AcidTable AcidTables} that can receive mutation events under the control of this
+ * client.
+ */
+ public List<AcidTable> getTables() throws ConnectionException {
+ if (!connected) {
+ throw new ConnectionException("Not connected - cannot interrogate tables.");
+ }
+ return Collections.<AcidTable> unmodifiableList(tables);
+ }
+
+ @Override
+ public String toString() {
+ return "MutatorClient [metaStoreClient=" + metaStoreClient + ", connected=" + connected + "]";
+ }
+
+ private void checkTable(IMetaStoreClient metaStoreClient, AcidTable acidTable) throws ConnectionException {
+ try {
+ LOG.debug("Checking table {}.", acidTable.getQualifiedName());
+ Table metaStoreTable = metaStoreClient.getTable(acidTable.getDatabaseName(), acidTable.getTableName());
+
+ if (acidTable.getTableType() == TableType.SINK) {
+ Map<String, String> parameters = metaStoreTable.getParameters();
+ if (!Boolean.parseBoolean(parameters.get(TRANSACTIONAL_PARAM_KEY))) {
+ throw new ConnectionException("Cannot stream to table that is not transactional: '"
+ + acidTable.getQualifiedName() + "'.");
+ }
+ int totalBuckets = metaStoreTable.getSd().getNumBuckets();
+ LOG.debug("Table {} has {} buckets.", acidTable.getQualifiedName(), totalBuckets);
+ if (totalBuckets <= 0) {
+ throw new ConnectionException("Cannot stream to table that has not been bucketed: '"
+ + acidTable.getQualifiedName() + "'.");
+ }
+
+ String outputFormat = metaStoreTable.getSd().getOutputFormat();
+ LOG.debug("Table {} has {} OutputFormat.", acidTable.getQualifiedName(), outputFormat);
+ acidTable.setTable(metaStoreTable);
+ }
+ } catch (NoSuchObjectException e) {
+ throw new ConnectionException("Invalid table '" + acidTable.getQualifiedName() + "'", e);
+ } catch (TException e) {
+ throw new ConnectionException("Error communicating with the meta store", e);
+ }
+ LOG.debug("Table {} OK.", acidTable.getQualifiedName());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java
new file mode 100644
index 0000000..6c21c59
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/MutatorClientBuilder.java
@@ -0,0 +1,115 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.hcatalog.common.HCatUtil;
+import org.apache.hive.hcatalog.streaming.mutate.HiveConfFactory;
+import org.apache.hive.hcatalog.streaming.mutate.UgiMetaStoreClientFactory;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockFailureListener;
+
+/** Convenience class for building {@link MutatorClient} instances. */
+public class MutatorClientBuilder {
+
+ private final Map<String, AcidTable> tables = new HashMap<>();
+ private HiveConf configuration;
+ private UserGroupInformation authenticatedUser;
+ private String metaStoreUri;
+ public LockFailureListener lockFailureListener;
+
+ public MutatorClientBuilder configuration(HiveConf conf) {
+ this.configuration = conf;
+ return this;
+ }
+
+ public MutatorClientBuilder authenticatedUser(UserGroupInformation authenticatedUser) {
+ this.authenticatedUser = authenticatedUser;
+ return this;
+ }
+
+ public MutatorClientBuilder metaStoreUri(String metaStoreUri) {
+ this.metaStoreUri = metaStoreUri;
+ return this;
+ }
+
+ /** Set a listener to handle {@link Lock} failure events - highly recommended. */
+ public MutatorClientBuilder lockFailureListener(LockFailureListener lockFailureListener) {
+ this.lockFailureListener = lockFailureListener;
+ return this;
+ }
+
+ /**
+ * Adds a mutation event destination (an ACID table) to be managed by this client, which is either unpartitioned or
+ * will is not to have partitions created automatically.
+ */
+ public MutatorClientBuilder addSourceTable(String databaseName, String tableName) {
+ addTable(databaseName, tableName, false, TableType.SOURCE);
+ return this;
+ }
+
+ /**
+ * Adds a mutation event destination (an ACID table) to be managed by this client, which is either unpartitioned or
+ * will is not to have partitions created automatically.
+ */
+ public MutatorClientBuilder addSinkTable(String databaseName, String tableName) {
+ return addSinkTable(databaseName, tableName, false);
+ }
+
+ /**
+ * Adds a partitioned mutation event destination (an ACID table) to be managed by this client, where new partitions
+ * will be created as needed.
+ */
+ public MutatorClientBuilder addSinkTable(String databaseName, String tableName, boolean createPartitions) {
+ addTable(databaseName, tableName, createPartitions, TableType.SINK);
+ return this;
+ }
+
+ private void addTable(String databaseName, String tableName, boolean createPartitions, TableType tableType) {
+ if (databaseName == null) {
+ throw new IllegalArgumentException("Database cannot be null");
+ }
+ if (tableName == null) {
+ throw new IllegalArgumentException("Table cannot be null");
+ }
+ String key = (databaseName + "." + tableName).toUpperCase();
+ AcidTable previous = tables.get(key);
+ if (previous != null) {
+ if (tableType == TableType.SINK && previous.getTableType() != TableType.SINK) {
+ tables.remove(key);
+ } else {
+ throw new IllegalArgumentException("Table has already been added: " + databaseName + "." + tableName);
+ }
+ }
+
+ Table table = new Table();
+ table.setDbName(databaseName);
+ table.setTableName(tableName);
+ tables.put(key, new AcidTable(databaseName, tableName, createPartitions, tableType));
+ }
+
+ /** Builds the client. */
+ public MutatorClient build() throws ClientException, MetaException {
+ String user = authenticatedUser == null ? System.getProperty("user.name") : authenticatedUser.getShortUserName();
+ boolean secureMode = authenticatedUser == null ? false : authenticatedUser.hasKerberosCredentials();
+
+ configuration = HiveConfFactory.newInstance(configuration, this.getClass(), metaStoreUri);
+
+ IMetaStoreClient metaStoreClient;
+ try {
+ metaStoreClient = new UgiMetaStoreClientFactory(metaStoreUri, configuration, authenticatedUser, user, secureMode)
+ .newInstance(HCatUtil.getHiveMetastoreClient(configuration));
+ } catch (IOException e) {
+ throw new ClientException("Could not create meta store client.", e);
+ }
+
+ return new MutatorClient(metaStoreClient, configuration, lockFailureListener, user, tables.values());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java
new file mode 100644
index 0000000..aa6d239
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TableType.java
@@ -0,0 +1,37 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+public enum TableType {
+ SOURCE((byte) 0),
+ SINK((byte) 1);
+
+ private static final TableType[] INDEX = buildIndex();
+
+ private static TableType[] buildIndex() {
+ TableType[] index = new TableType[TableType.values().length];
+ for (TableType type : values()) {
+ byte position = type.getId();
+ if (index[position] != null) {
+ throw new IllegalStateException("Overloaded index: " + position);
+ }
+ index[position] = type;
+ }
+ return index;
+ }
+
+ private byte id;
+
+ private TableType(byte id) {
+ this.id = id;
+ }
+
+ public byte getId() {
+ return id;
+ }
+
+ public static TableType valueOf(byte id) {
+ if (id < 0 || id >= INDEX.length) {
+ throw new IllegalArgumentException("Invalid id: " + id);
+ }
+ return INDEX[id];
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java
new file mode 100644
index 0000000..6532900
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/Transaction.java
@@ -0,0 +1,114 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockException;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class Transaction {
+
+ private static final Logger LOG = LoggerFactory.getLogger(Transaction.class);
+
+ private final Lock lock;
+ private final IMetaStoreClient metaStoreClient;
+ private final long transactionId;
+
+ private TxnState state;
+
+ Transaction(IMetaStoreClient metaStoreClient, Lock.Options lockOptions) throws TransactionException {
+ this(metaStoreClient, new Lock(metaStoreClient, lockOptions));
+ }
+
+ /** Visible for testing only. */
+ Transaction(IMetaStoreClient metaStoreClient, Lock lock) throws TransactionException {
+ this.metaStoreClient = metaStoreClient;
+ this.lock = lock;
+ transactionId = open(lock.getUser());
+ }
+
+ public long getTransactionId() {
+ return transactionId;
+ }
+
+ public TxnState getState() {
+ return state;
+ }
+
+ /**
+ * Begin the transaction. Acquires a {@link Lock} for the transaction and {@link AcidTable AcidTables}.
+ */
+ public void begin() throws TransactionException {
+ try {
+ lock.acquire(transactionId);
+ } catch (LockException e) {
+ throw new TransactionException("Unable to acquire lock for transaction: " + transactionId, e);
+ }
+ state = TxnState.OPEN;
+ LOG.debug("Begin. Transaction id: {}", transactionId);
+ }
+
+ /** Commits the transaction. Releases the {@link Lock}. */
+ public void commit() throws TransactionException {
+ try {
+ lock.release();
+ } catch (LockException e) {
+ // This appears to leave the remove transaction in an inconsistent state but the heartbeat is now
+ // cancelled and it will eventually time out
+ throw new TransactionException("Unable to release lock: " + lock + " for transaction: " + transactionId, e);
+ }
+ try {
+ metaStoreClient.commitTxn(transactionId);
+ state = TxnState.COMMITTED;
+ } catch (NoSuchTxnException e) {
+ throw new TransactionException("Invalid transaction id: " + transactionId, e);
+ } catch (TxnAbortedException e) {
+ throw new TransactionException("Aborted transaction cannot be committed: " + transactionId, e);
+ } catch (TException e) {
+ throw new TransactionException("Unable to commit transaction: " + transactionId, e);
+ }
+ LOG.debug("Committed. Transaction id: {}", transactionId);
+ }
+
+ /** Aborts the transaction. Releases the {@link Lock}. */
+ public void abort() throws TransactionException {
+ try {
+ lock.release();
+ } catch (LockException e) {
+ // This appears to leave the remove transaction in an inconsistent state but the heartbeat is now
+ // cancelled and it will eventually time out
+ throw new TransactionException("Unable to release lock: " + lock + " for transaction: " + transactionId, e);
+ }
+ try {
+ metaStoreClient.rollbackTxn(transactionId);
+ state = TxnState.ABORTED;
+ } catch (NoSuchTxnException e) {
+ throw new TransactionException("Unable to abort invalid transaction id : " + transactionId, e);
+ } catch (TException e) {
+ throw new TransactionException("Unable to abort transaction id : " + transactionId, e);
+ }
+ LOG.debug("Aborted. Transaction id: {}", transactionId);
+ }
+
+ @Override
+ public String toString() {
+ return "Transaction [transactionId=" + transactionId + ", state=" + state + "]";
+ }
+
+ private long open(String user) throws TransactionException {
+ long transactionId = -1;
+ try {
+ transactionId = metaStoreClient.openTxn(user);
+ state = TxnState.INACTIVE;
+ } catch (TException e) {
+ throw new TransactionException("Unable to open transaction for user: " + user, e);
+ }
+ LOG.debug("Opened transaction with id: {}", transactionId);
+ return transactionId;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java
new file mode 100644
index 0000000..48fb1cf
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/TransactionException.java
@@ -0,0 +1,15 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+public class TransactionException extends ClientException {
+
+ private static final long serialVersionUID = 1L;
+
+ TransactionException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ TransactionException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java
new file mode 100644
index 0000000..5814d37
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatFactory.java
@@ -0,0 +1,30 @@
+package org.apache.hive.hcatalog.streaming.mutate.client.lock;
+
+import java.util.Collection;
+import java.util.Timer;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Creates a default {@link HeartbeatTimerTask} for {@link Lock Locks}. */
+class HeartbeatFactory {
+
+ private static final Logger LOG = LoggerFactory.getLogger(HeartbeatFactory.class);
+
+ /** Creates a new {@link HeartbeatTimerTask} instance for the {@link Lock} and schedules it. */
+ Timer newInstance(IMetaStoreClient metaStoreClient, LockFailureListener listener, Long transactionId,
+ Collection<Table> tableDescriptors, long lockId, int heartbeatPeriod) {
+ Timer heartbeatTimer = new Timer("hive-lock-heartbeat[lockId=" + lockId + ", transactionId=" + transactionId + "]",
+ true);
+ HeartbeatTimerTask task = new HeartbeatTimerTask(metaStoreClient, listener, transactionId, tableDescriptors, lockId);
+ heartbeatTimer.schedule(task, TimeUnit.SECONDS.toMillis(heartbeatPeriod),
+ TimeUnit.SECONDS.toMillis(heartbeatPeriod));
+
+ LOG.debug("Scheduled heartbeat timer task: {}", heartbeatTimer);
+ return heartbeatTimer;
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java
new file mode 100644
index 0000000..2446c10
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/HeartbeatTimerTask.java
@@ -0,0 +1,66 @@
+package org.apache.hive.hcatalog.streaming.mutate.client.lock;
+
+import java.util.Collection;
+import java.util.TimerTask;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.hive.hcatalog.streaming.mutate.client.Transaction;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link TimerTask} that sends {@link IMetaStoreClient#heartbeat(long, long) heartbeat} events to the
+ * {@link IMetaStoreClient meta store} to keet the {@link Lock} and {@link Transaction} alive. Nofifies the registered
+ * {@link LockFailureListener} should the lock fail.
+ */
+class HeartbeatTimerTask extends TimerTask {
+
+ private static final Logger LOG = LoggerFactory.getLogger(HeartbeatTimerTask.class);
+
+ private final IMetaStoreClient metaStoreClient;
+ private final long lockId;
+ private final Long transactionId;
+ private final LockFailureListener listener;
+ private final Collection<Table> tableDescriptors;
+
+ HeartbeatTimerTask(IMetaStoreClient metaStoreClient, LockFailureListener listener, Long transactionId,
+ Collection<Table> tableDescriptors, long lockId) {
+ this.metaStoreClient = metaStoreClient;
+ this.listener = listener;
+ this.transactionId = transactionId;
+ this.tableDescriptors = tableDescriptors;
+ this.lockId = lockId;
+ LOG.debug("Reporting to listener {}", listener);
+ }
+
+ @Override
+ public void run() {
+ try {
+ // I'm assuming that there is no transaction ID for a read lock.
+ metaStoreClient.heartbeat(transactionId == null ? 0 : transactionId, lockId);
+ LOG.debug("Sent heartbeat for lock={}, transactionId={}", lockId, transactionId);
+ } catch (NoSuchLockException | NoSuchTxnException | TxnAbortedException e) {
+ failLock(e);
+ } catch (TException e) {
+ LOG.warn("Failed to send heartbeat to meta store.", e);
+ }
+ }
+
+ private void failLock(Exception e) {
+ LOG.debug("Lock " + lockId + " failed, cancelling heartbeat and notifiying listener: " + listener, e);
+ // Cancel the heartbeat
+ cancel();
+ listener.lockFailed(lockId, transactionId, Lock.asStrings(tableDescriptors), e);
+ }
+
+ @Override
+ public String toString() {
+ return "HeartbeatTimerTask [lockId=" + lockId + ", transactionId=" + transactionId + "]";
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
new file mode 100644
index 0000000..21604df
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/Lock.java
@@ -0,0 +1,282 @@
+package org.apache.hive.hcatalog.streaming.mutate.client.lock;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.Timer;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.LockComponentBuilder;
+import org.apache.hadoop.hive.metastore.LockRequestBuilder;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockState;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Manages the state required to safely read/write from/to an ACID table.
+ */
+public class Lock {
+
+ private static final Logger LOG = LoggerFactory.getLogger(Lock.class);
+
+ private static final double HEARTBEAT_FACTOR = 0.75;
+ private static final int DEFAULT_HEARTBEAT_PERIOD = 275;
+
+ private final IMetaStoreClient metaStoreClient;
+ private final HeartbeatFactory heartbeatFactory;
+ private final LockFailureListener listener;
+ private final Collection<Table> tableDescriptors;
+ private final int lockRetries;
+ private final int retryWaitSeconds;
+ private final String user;
+ private final HiveConf hiveConf;
+
+ private Timer heartbeat;
+ private Long lockId;
+ private Long transactionId;
+
+ public Lock(IMetaStoreClient metaStoreClient, Options options) {
+ this(metaStoreClient, new HeartbeatFactory(), options.hiveConf, options.listener, options.user,
+ options.descriptors, options.lockRetries, options.retryWaitSeconds);
+ }
+
+ /** Visible for testing only. */
+ Lock(IMetaStoreClient metaStoreClient, HeartbeatFactory heartbeatFactory, HiveConf hiveConf,
+ LockFailureListener listener, String user, Collection<Table> tableDescriptors, int lockRetries,
+ int retryWaitSeconds) {
+ this.metaStoreClient = metaStoreClient;
+ this.heartbeatFactory = heartbeatFactory;
+ this.hiveConf = hiveConf;
+ this.user = user;
+ this.tableDescriptors = tableDescriptors;
+ this.listener = listener;
+ this.lockRetries = lockRetries;
+ this.retryWaitSeconds = retryWaitSeconds;
+
+ if (LockFailureListener.NULL_LISTENER.equals(listener)) {
+ LOG.warn("No {} supplied. Data quality and availability cannot be assured.",
+ LockFailureListener.class.getSimpleName());
+ }
+ }
+
+ /** Attempts to acquire a read lock on the table, returns if successful, throws exception otherwise. */
+ public void acquire() throws LockException {
+ lockId = internalAcquire(null);
+ initiateHeartbeat();
+ }
+
+ /** Attempts to acquire a read lock on the table, returns if successful, throws exception otherwise. */
+ public void acquire(long transactionId) throws LockException {
+ lockId = internalAcquire(transactionId);
+ this.transactionId = transactionId;
+ initiateHeartbeat();
+ }
+
+ /** Attempts to release the read lock on the table. Throws an exception if the lock failed at any point. */
+ public void release() throws LockException {
+ if (heartbeat != null) {
+ heartbeat.cancel();
+ }
+ internalRelease();
+ }
+
+ public String getUser() {
+ return user;
+ }
+
+ @Override
+ public String toString() {
+ return "Lock [metaStoreClient=" + metaStoreClient + ", lockId=" + lockId + ", transactionId=" + transactionId
+ + "]";
+ }
+
+ private long internalAcquire(Long transactionId) throws LockException {
+ int attempts = 0;
+ LockRequest request = buildSharedLockRequest(transactionId);
+ do {
+ LockResponse response = null;
+ try {
+ response = metaStoreClient.lock(request);
+ } catch (TException e) {
+ throw new LockException("Unable to acquire lock for tables: [" + join(tableDescriptors) + "]", e);
+ }
+ if (response != null) {
+ LockState state = response.getState();
+ if (state == LockState.NOT_ACQUIRED || state == LockState.ABORT) {
+ // I expect we'll only see NOT_ACQUIRED here?
+ break;
+ }
+ if (state == LockState.ACQUIRED) {
+ LOG.debug("Acquired lock {}", response.getLockid());
+ return response.getLockid();
+ }
+ if (state == LockState.WAITING) {
+ try {
+ Thread.sleep(TimeUnit.SECONDS.toMillis(retryWaitSeconds));
+ } catch (InterruptedException e) {
+ }
+ }
+ }
+ attempts++;
+ } while (attempts < lockRetries);
+ throw new LockException("Could not acquire lock on tables: [" + join(tableDescriptors) + "]");
+ }
+
+ private void internalRelease() {
+ try {
+ // if there is a transaction then this lock will be released on commit/abort/rollback instead.
+ if (lockId != null && transactionId == null) {
+ metaStoreClient.unlock(lockId);
+ LOG.debug("Released lock {}", lockId);
+ lockId = null;
+ }
+ } catch (TException e) {
+ LOG.error("Lock " + lockId + " failed.", e);
+ listener.lockFailed(lockId, transactionId, asStrings(tableDescriptors), e);
+ }
+ }
+
+ private LockRequest buildSharedLockRequest(Long transactionId) {
+ LockRequestBuilder requestBuilder = new LockRequestBuilder();
+ for (Table descriptor : tableDescriptors) {
+ LockComponent component = new LockComponentBuilder()
+ .setDbName(descriptor.getDbName())
+ .setTableName(descriptor.getTableName())
+ .setShared()
+ .build();
+ requestBuilder.addLockComponent(component);
+ }
+ if (transactionId != null) {
+ requestBuilder.setTransactionId(transactionId);
+ }
+ LockRequest request = requestBuilder.setUser(user).build();
+ return request;
+ }
+
+ private void initiateHeartbeat() {
+ int heartbeatPeriod = getHeartbeatPeriod();
+ LOG.debug("Heartbeat period {}s", heartbeatPeriod);
+ heartbeat = heartbeatFactory.newInstance(metaStoreClient, listener, transactionId, tableDescriptors, lockId,
+ heartbeatPeriod);
+ }
+
+ private int getHeartbeatPeriod() {
+ int heartbeatPeriod = DEFAULT_HEARTBEAT_PERIOD;
+ if (hiveConf != null) {
+ // This value is always in seconds and includes an 's' suffix.
+ String txTimeoutSeconds = hiveConf.getVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT);
+ if (txTimeoutSeconds != null) {
+ // We want to send the heartbeat at an interval that is less than the timeout.
+ heartbeatPeriod = Math.max(1,
+ (int) (Integer.parseInt(txTimeoutSeconds.substring(0, txTimeoutSeconds.length() - 1)) * HEARTBEAT_FACTOR));
+ }
+ }
+ return heartbeatPeriod;
+ }
+
+ /** Visible for testing only. */
+ Long getLockId() {
+ return lockId;
+ }
+
+ /** Visible for testing only. */
+ Long getTransactionId() {
+ return transactionId;
+ }
+
+ /** Visible for testing only. */
+ static String join(Iterable<? extends Object> values) {
+ return StringUtils.join(values, ",");
+ }
+
+ /** Visible for testing only. */
+ static List<String> asStrings(Collection<Table> tables) {
+ List<String> strings = new ArrayList<>(tables.size());
+ for (Table descriptor : tables) {
+ strings.add(descriptor.getDbName() + "." + descriptor.getTableName());
+ }
+ return strings;
+ }
+
+ /** Constructs a lock options for a set of Hive ACID tables from which we wish to read. */
+ public static final class Options {
+ Set<Table> descriptors = new LinkedHashSet<>();
+ LockFailureListener listener = LockFailureListener.NULL_LISTENER;
+ int lockRetries = 5;
+ int retryWaitSeconds = 30;
+ String user;
+ HiveConf hiveConf;
+
+ /** Adds a table for which a shared read lock will be requested. */
+ public Options addTable(String databaseName, String tableName) {
+ checkNotNullOrEmpty(databaseName);
+ checkNotNullOrEmpty(tableName);
+ Table table = new Table();
+ table.setDbName(databaseName);
+ table.setTableName(tableName);
+ descriptors.add(table);
+ return this;
+ }
+
+ public Options user(String user) {
+ checkNotNullOrEmpty(user);
+ this.user = user;
+ return this;
+ }
+
+ public Options configuration(HiveConf hiveConf) {
+ checkNotNull(hiveConf);
+ this.hiveConf = hiveConf;
+ return this;
+ }
+
+ /** Sets a listener to handle failures of locks that were previously acquired. */
+ public Options lockFailureListener(LockFailureListener listener) {
+ checkNotNull(listener);
+ this.listener = listener;
+ return this;
+ }
+
+ public Options lockRetries(int lockRetries) {
+ checkArgument(lockRetries > 0);
+ this.lockRetries = lockRetries;
+ return this;
+ }
+
+ public Options retryWaitSeconds(int retryWaitSeconds) {
+ checkArgument(retryWaitSeconds > 0);
+ this.retryWaitSeconds = retryWaitSeconds;
+ return this;
+ }
+
+ private static void checkArgument(boolean value) {
+ if (!value) {
+ throw new IllegalArgumentException();
+ }
+ }
+
+ private static void checkNotNull(Object value) {
+ if (value == null) {
+ throw new IllegalArgumentException();
+ }
+ }
+
+ private static void checkNotNullOrEmpty(String value) {
+ if (StringUtils.isBlank(value)) {
+ throw new IllegalArgumentException();
+ }
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java
new file mode 100644
index 0000000..67ed601
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockException.java
@@ -0,0 +1,15 @@
+package org.apache.hive.hcatalog.streaming.mutate.client.lock;
+
+public class LockException extends Exception {
+
+ private static final long serialVersionUID = 1L;
+
+ public LockException(String message) {
+ super(message);
+ }
+
+ public LockException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java
new file mode 100644
index 0000000..2b6a12a
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/client/lock/LockFailureListener.java
@@ -0,0 +1,26 @@
+package org.apache.hive.hcatalog.streaming.mutate.client.lock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Provides a means to handle the situation when a held lock fails. */
+public interface LockFailureListener {
+
+ static final Logger LOG = LoggerFactory.getLogger(LockFailureListener.class);
+
+ static final LockFailureListener NULL_LISTENER = new LockFailureListener() {
+ @Override
+ public void lockFailed(long lockId, Long transactionId, Iterable<String> tableNames, Throwable t) {
+ LOG.warn(
+ "Ignored lock failure: lockId=" + lockId + ", transactionId=" + transactionId + ", tables=" + tableNames, t);
+ }
+
+ public String toString() {
+ return LockFailureListener.class.getName() + ".NULL_LISTENER";
+ }
+ };
+
+ /** Called when the specified lock has failed. You should probably abort your job in this case. */
+ void lockFailed(long lockId, Long transactionId, Iterable<String> tableNames, Throwable t);
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot
new file mode 100644
index 0000000..79c30e7
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/doc-files/system-overview.dot
@@ -0,0 +1,27 @@
+digraph "API Usage" {
+ nodesep=1.2;
+
+ DATA [label="ACID\ndataset",shape=oval,style=filled,color="gray"];
+ CHANGES [label="Changed\ndata",shape=oval,style=filled,color="gray"];
+
+ META_STORE [label="Hive\nMetaStore",shape=box,style=filled,color="darkseagreen3"];
+ HIVE_CLI [label="Hive\nCLI",shape=box,style=filled,color="darkseagreen3"];
+
+ MERGE1 [label="Compute\nmutations\n(your code)",shape=box,style=filled,color="khaki1"];
+ SORT [label="Group\n& sort\n(your code)",shape=box,style=filled,color="khaki1"];
+ CLIENT [label="Mutator\nclient",shape=box,style=filled,color="lightblue"];
+ BUCKET [label="Bucket ID\nappender",shape=box,style=filled,color="lightblue"];
+ COORD [label="Mutator\ncoordinator",shape=box,style=filled,color="lightblue"];
+ CLIENT -> COORD [label="Provides\nconf to"];
+ CLIENT -> BUCKET [label="Provides\nconf to"];
+
+ CLIENT -> META_STORE [label="Manages\ntxns using"];
+ CHANGES -> MERGE1 [label="Reads ∆s\nfrom"];
+ DATA -> MERGE1 [label="Reads\nROW__IDs\nfrom"];
+ BUCKET -> MERGE1 [label="Appends ids\nto inserts"];
+ MERGE1 -> SORT;
+ SORT -> COORD [label="Issues\nmutations to"];
+ COORD -> DATA [label="Writes to"];
+ DATA -> HIVE_CLI [label="Read by"];
+ META_STORE -> DATA [label="Compacts"];
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
new file mode 100644
index 0000000..9fc10b6
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/package.html
@@ -0,0 +1,495 @@
+<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN"
+ "http://www.w3.org/TR/html4/loose.dtd">
+
+<html lang="en">
+
+<head>
+<meta name=Title content="HCatalog Streaming Mutation API">
+<meta name=Keywords content="HCatalog Streaming Mutation ACID">
+<meta http-equiv=Content-Type content="text/html; charset=utf-8">
+<title>HCatalog Streaming Mutation API</title>
+</head>
+
+<body>
+
+<h1>HCatalog Streaming Mutation API -- high level description</h1>
+
+<h2>Background</h2>
+<p>
+In certain data processing use cases it is necessary to modify existing
+data when new facts arrive. An example of this is the classic ETL merge
+where a copy of a data set is kept in sync with a master by the frequent
+application of deltas. The deltas describe the mutations (inserts,
+updates, deletes) that have occurred to the master since the previous
+sync. To implement such a case using Hadoop traditionally demands that
+the partitions containing records targeted by the mutations be
+rewritten. This is a coarse approach; a partition containing millions of
+records might be rebuilt because of a single record change. Additionally
+these partitions cannot be restated atomically; at some point the old
+partition data must be swapped with the new partition data. When this
+swap occurs, usually by issuing an HDFS
+<code>rm</code>
+followed by a
+<code>mv</code>
+, the possibility exists where the data appears to be unavailable and
+hence any downstream jobs consuming the data might unexpectedly fail.
+Therefore data processing patterns that restate raw data on HDFS cannot
+operate robustly without some external mechanism to orchestrate
+concurrent access to changing data.
+</p>
+
+<p>
+The availability of ACID tables in Hive provides a mechanism that both
+enables concurrent access to data stored in HDFS (so long as it's in the
+ORC+ACID format), and also permits row level mutations or records within
+a table, without the need to rewrite the existing data. But while Hive
+itself supports
+<code>INSERT</code>
+,
+<code>UPDATE</code>
+and
+<code>DELETE</code>
+commands, and the ORC format can support large batches of mutations in a
+transaction, Hive's execution engine currently submits each individual
+mutation operation in a separate transaction and issues table scans (M/R
+jobs) to execute them. It does not currently scale to the demands of
+processing large deltas in an atomic manner. Furthermore it would be
+advantageous to extend atomic batch mutation capabilities beyond Hive by
+making them available to other data processing frameworks. The Streaming
+Mutation API does just this.
+</p>
+
+<p>The Streaming Mutation API, although similar to the Streaming
+API, has a number of differences and are built to enable very different
+use cases. Superficially, the Streaming API can only write new data
+whereas the mutation API can also modify existing data. However the two
+APIs also based on very different transaction models. The Streaming API
+focuses on surfacing a continuous stream of new data into a Hive table
+and does so by batching small sets of writes into multiple short-lived
+transactions. Conversely the mutation API is designed to infrequently
+apply large sets of mutations to a data set in an atomic fashion; all
+mutations will either be applied or they will not. This instead mandates
+the use of a single long-lived transaction. This table summarises the
+attributes of each API:</p>
+
+<table border="1">
+<thead>
+<tr>
+<th>Attribute</th>
+<th>Streaming API</th>
+<th>Mutation API</th>
+</tr>
+<tr>
+<td>Ingest type</td>
+<td>Data arrives continuously</td>
+<td>Ingests are performed periodically and the mutations are
+applied in a single batch</td>
+</tr>
+<tr>
+<td>Transaction scope</td>
+<td>Transactions are created for small batches of writes</td>
+<td>The entire set of mutations should be applied within a single
+transaction</td>
+</tr>
+<tr>
+<td>Data availability</td>
+<td>Surfaces new data to users frequently and quickly</td>
+<td>Change sets should be applied atomically, either the effect of
+the delta is visible or it is not</td>
+</tr>
+<tr>
+<td>Sensitive to record order</td>
+<td>No, records do not have pre-existing lastTxnIds or bucketIds.
+Records are likely being written into a single partition (today's date
+for example)</td>
+<td>Yes, all mutated records have existing <code>RecordIdentifiers</code>
+and must be grouped by (partitionValues, bucketId) and sorted by
+lastTxnId. These record coordinates initially arrive in an order that is
+effectively random.
+</td>
+</tr>
+<tr>
+<td>Impact of a write failure</td>
+<td>Transaction can be aborted and producer can choose to resubmit
+failed records as ordering is not important.</td>
+<td>Ingest for the respective must be halted and failed records
+resubmitted to preserve sequence.</td>
+</tr>
+<tr>
+<td>User perception of missing data</td>
+<td>Data has not arrived yet → "latency?"</td>
+<td>"This data is inconsistent, some records have been updated, but
+other related records have not" - consider here the classic transfer
+between bank accounts scenario</td>
+</tr>
+<tr>
+<td>API end point scope</td>
+<td>A given <code>HiveEndPoint</code> instance submits many
+transactions to a specific bucket, in a specific partition, of a
+specific table
+</td>
+<td>A set of<code>MutationCoordinators</code> write changes to
+unknown set of buckets, of an unknown set of partitions, of specific
+tables (can be more than one), within a single transaction.
+</td>
+</tr>
+</thead>
+</table>
+
+<h2>Structure</h2>
+<p>The API comprises two main concerns: transaction management, and
+the writing of mutation operations to the data set. The two concerns
+have a minimal coupling as it is expected that transactions will be
+initiated from a single job launcher type processes while the writing of
+mutations will be scaled out across any number of worker nodes. In the
+context of Hadoop M/R these can be more concretely defined as the Tool
+and Map/Reduce task components. However, use of this architecture is not
+mandated and in fact both concerns could be handled within a single
+simple process depending on the requirements.</p>
+
+<p>Note that a suitably configured Hive instance is required to
+operate this system even if you do not intend to access the data from
+within Hive. Internally, transactions are managed by the Hive MetaStore.
+Mutations are performed to HDFS via ORC APIs that bypass the MetaStore.
+Additionally you may wish to configure your MetaStore instance to
+perform periodic data compactions.</p>
+
+<p>
+<b>Note on packaging</b>: The APIs are defined in the <b>org.apache.hive.hcatalog.streaming.mutate</b>
+Java package and included as the hive-hcatalog-streaming jar.
+</p>
+
+<h2>Data requirements</h2>
+<p>
+Generally speaking, to apply a mutation to a record one must have some
+unique key that identifies the record. However, primary keys are not a
+construct provided by Hive. Internally Hive uses
+<code>RecordIdentifiers</code>
+stored in a virtual
+<code>ROW__ID</code>
+column to uniquely identified records within an ACID table. Therefore,
+any process that wishes to issue mutations to a table via this API must
+have available the corresponding row ids for the target records. What
+this means in practice is that the process issuing mutations must first
+read in a current snapshot the data and then join the mutations on some
+domain specific primary key to obtain the corresponding Hive
+<code>ROW__ID</code>
+. This is effectively what occurs within Hive's table scan process when
+an
+<code>UPDATE</code>
+or
+<code>DELETE</code>
+statement is executed. The
+<code>AcidInputFormat</code>
+provides access to this data via
+<code>AcidRecordReader.getRecordIdentifier()</code>
+.
+</p>
+
+<p>
+The implementation of the ACID format places some constraints on the
+order in which records are written and it is important that this
+ordering is enforced. Additionally, data must be grouped appropriately
+to adhere to the constraints imposed be the
+<code>OrcRecordUpdater</code>
+. Grouping also makes it possible parallelise the writing of mutations
+for the purposes of scaling. Finally, to correctly bucket new records
+(inserts) there is a slightly unintuitive trick that must be applied.
+</p>
+
+<p>All of these data sequencing concerns are the responsibility of
+the client process calling the API which is assumed to have first class
+grouping and sorting capabilities (Hadoop Map/Reduce etc.) The streaming
+API provides nothing more than validators that fail fast when they
+encounter groups and records that are out of sequence.</p>
+
+<p>In short, API client processes should prepare data for the mutate
+API like so:</p>
+<ul>
+<li><b>MUST:</b> Order records by <code>ROW__ID.originalTxn</code>,
+then <code>ROW__ID.rowId</code>.</li>
+<li><b>MUST:</b> Assign a <code>ROW__ID</code> containing a
+computed <code>bucketId</code> to records to be inserted.</li>
+<li><b>SHOULD:</b> Group/partition by table partition value, then <code>ROW__ID.bucketId</code>.</li>
+</ul>
+
+<p>
+The addition of a bucket ids to insert records prior to grouping and
+sorting seems unintuitive. However, it is required both to ensure
+adequate partitioning of new data and bucket allocation consistent with
+that provided by Hive. In a typical ETL the majority of mutation events
+are inserts, often targeting a single partition (new data for the
+previous day, hour, etc.) If more that one worker is writing said
+events, were we to leave the bucket id empty then all inserts would go
+to a single worker (e.g: reducer) and the workload could be heavily
+skewed. The assignment of a computed bucket allows inserts to be more
+usefully distributed across workers. Additionally, when Hive is working
+with the data it may expect records to have been bucketed in a way that
+is consistent with it's own internal scheme. A convenience type and
+method is provided to more easily compute and append bucket ids:
+<code>BucketIdResolver</code>
+and
+<code>BucketIdResolverImpl</code>
+.
+</p>
+
+<p>Update operations should not attempt to modify values of
+partition or bucketing columns. The API does not prevent this and such
+attempts could lead to data corruption.</p>
+
+<h2>Streaming requirements</h2>
+<p>A few things are currently required to use streaming.</p>
+
+<p>
+<ol>
+<li>Currently, only ORC storage format is supported. So '<b>stored
+as orc</b>' must be specified during table creation.
+</li>
+<li>The hive table must be bucketed, but not sorted. So something
+like '<b>clustered by (<i>colName</i>) into <i>10</i> buckets
+</b>' must be specified during table creation.
+</li>
+<li>User of the client streaming process must have the necessary
+permissions to write to the table or partition and create partitions in
+the table.</li>
+<li>Settings required in hive-site.xml for Metastore:
+<ol>
+<li><b>hive.txn.manager =
+org.apache.hadoop.hive.ql.lockmgr.DbTxnManager</b></li>
+<li><b>hive.support.concurrency = true </b></li>
+<li><b>hive.compactor.initiator.on = true</b></li>
+<li><b>hive.compactor.worker.threads > 0 </b></li>
+</ol>
+</li>
+</ol>
+</p>
+
+<p>
+<b>Note:</b> Streaming mutations to <b>unpartitioned</b> tables is also
+supported.
+</p>
+
+<h2>Record layout</h2>
+<p>
+The structure, layout, and encoding of records is the exclusive concern
+of the client ETL mutation process and may be quite different from the
+target Hive ACID table. The mutation API requires concrete
+implementations of the
+<code>MutatorFactory</code>
+and
+<code>Mutator</code>
+classes to extract pertinent data from records and serialize data into
+the ACID files. Fortunately base classes are provided (
+<code>AbstractMutator</code>
+,
+<code>RecordInspectorImpl</code>
+) to simplify this effort and usually all that is required is the
+specification of a suitable
+<code>ObjectInspector</code>
+and the provision of the indexes of the
+<code>ROW__ID</code>
+and bucketed columns within the record structure. Note that all column
+indexes in these classes are with respect to your record structure, not
+the Hive table structure.
+</p>
+<p>
+You will likely also want to use a
+<code>BucketIdResolver</code>
+to append bucket ids to new records for insertion. Fortunately the core
+implementation is provided in
+<code>BucketIdResolverImpl</code>
+but note that bucket column indexes must be presented in the same order
+as they are in the Hive table definition to ensure consistent bucketing.
+Note that you cannot move records between buckets and an exception will
+be thrown if you attempt to do so. In real terms this mean that you
+should not attempt to modify the values in bucket columns with an
+<code>UPDATE</code>
+.
+</p>
+
+<h2>Connection and Transaction management</h2>
+<p>
+The
+<code>MutatorClient</code>
+class is used to create and manage transactions in which mutations can
+be performed. The scope of a transaction can extend across multiple ACID
+tables. When a client connects it communicates with the meta store to
+verify and acquire meta data for the target tables. An invocation of
+<code>newTransaction</code>
+then opens a transaction with the meta store, finalizes a collection of
+<code>AcidTables</code>
+and returns a new
+<code>Transaction</code>
+instance. The acid tables are light-weight, serializable objects that
+are used by the mutation writing components of the API to target
+specific ACID file locations. Usually your
+<code>MutatorClient</code>
+will be running on some master node and your coordinators on worker
+nodes. In this event the
+<code>AcidTableSerializer</code>
+can be used to encode the tables in a more transportable form, for use
+as a
+<code>Configuration</code>
+property for example.
+</p>
+<p>
+As you would expect, a
+<code>Transaction</code>
+must be initiated with a call to
+<code>begin</code>
+before any mutations can be applied. This invocation acquires a lock on
+the targeted tables using the meta store, and initiates a heartbeat to
+prevent transaction timeouts. It is highly recommended that you register
+a
+<code>LockFailureListener</code>
+with the client so that your process can handle any lock or transaction
+failures. Typically you may wish to abort the job in the event of such
+an error. With the transaction in place you can now start streaming
+mutations with one or more
+<code>MutatorCoordinator</code>
+instances (more on this later), can can finally
+<code>commit</code>
+or
+<code>abort</code>
+the transaction when the change set has been applied, which will release
+the lock with the meta store client. Finally you should
+<code>close</code>
+the mutation client to release any held resources.
+</p>
+<p>
+The
+<code>MutatorClientBuilder</code>
+is provided to simplify the construction of clients.
+</p>
+
+<p>
+<b>WARNING:</b> Hive doesn't currently have a deadlock detector (it is
+being worked on as part of <a
+href="https://issues.apache.org/jira/browse/HIVE-9675">HIVE-9675</a>).
+This API could potentially deadlock with other stream writers or with
+SQL users.
+</p>
+<h2>Writing data</h2>
+
+<p>
+The
+<code>MutatorCoordinator</code>
+class is used to issue mutations to an ACID table. You will require at
+least one instance per table participating in the transaction. The
+target of a given instance is defined by the respective
+<code>AcidTable</code>
+used to construct the coordinator. It is recommended that a
+<code>MutatorClientBuilder</code>
+is used to simplify the construction process.
+</p>
+
+<p>
+Mutations can be applied by invoking the respective
+<code>insert</code>
+,
+<code>update</code>
+, and
+<code>delete</code>
+methods on the coordinator. These methods each take as parameters the
+target partition of the record and the mutated record. In the case of an
+unpartitioned table you should simply pass an empty list as the
+partition value. For inserts specifically, only the bucket id will be
+extracted from the
+<code>RecordIdentifier</code>
+, the transactionId and rowId will be ignored and replaced by
+appropriate values in the
+<code>RecordUpdater</code>
+. Additionally, in the case of deletes, everything but the
+<code>RecordIdentifier</code>
+in the record will be ignored and therefore it is often easier to simply
+submit the original record.
+</p>
+
+<p>
+<b>Caution:</b> As mentioned previously, mutations must arrive in
+specific order for the resultant table data to be consistent.
+Coordinators will verify a naturally ordered sequence of
+(lastTransactionId, rowId) and will throw an exception if this sequence
+is broken. This exception should almost certainly be escalated so that
+the transaction is aborted. This, along with the correct ordering of the
+data, is the responsibility of the client using the API.
+</p>
+
+<h3>Dynamic Partition Creation:</h3>
+It is very likely to be desirable to have new partitions created
+automatically (say on a hourly basis). In such cases requiring the Hive
+admin to pre-create the necessary partitions may not be reasonable.
+Consequently the API allows coordinators to create partitions as needed
+(see:
+<code>MutatorClientBuilder.addTable(String, String, boolean)</code>
+). Partition creation being an atomic action, multiple coordinators can
+race to create the partition, but only one would succeed, so
+coordinators clients need not synchronize when creating a partition. The
+user of the coordinator process needs to be given write permissions on
+the Hive table in order to create partitions.
+
+<h2>Reading data</h2>
+
+<p>
+Although this API is concerned with writing changes to data, as
+previously stated we'll almost certainly have to read the existing data
+first to obtain the relevant
+<code>ROW_IDs</code>
+. Therefore it is worth noting that reading ACID data in a robust and
+consistent manner requires the following:
+<ol>
+<li>Obtaining a valid transaction list from the meta store (<code>ValidTxnList</code>).
+</li>
+<li>Acquiring a read-lock with the meta store and issuing
+heartbeats (<code>LockImpl</code> can help with this).
+</li>
+<li>Configuring the <code>OrcInputFormat</code> and then reading
+the data. Make sure that you also pull in the <code>ROW__ID</code>
+values. See: <code>AcidRecordReader.getRecordIdentifier</code>.
+</li>
+<li>Releasing the read-lock.</li>
+</ol>
+</p>
+
+<h2>Example</h2>
+<p>
+<img src="doc-files/system-overview.png" />
+</p>
+<p>So to recap, the sequence of events required to apply mutations
+to a dataset using the API is:</p>
+<ol>
+<li>Create a <code>MutatorClient</code> to manage a transaction for
+the targeted ACID tables. This set of tables should include any
+transactional destinations or sources. Don't forget to register a <code>LockFailureListener</code>
+so that you can handle transaction failures.
+</li>
+<li>Open a new <code>Transaction</code> with the client.
+</li>
+<li>Get the <code>AcidTables</code> from the client.
+</li>
+<li>Begin the transaction.</li>
+<li>Create at least one <code>MutatorCoordinator</code> for each
+table. The <code>AcidTableSerializer</code> can help you transport the <code>AcidTables</code>
+when your workers are in a distributed environment.
+</li>
+<li>Compute your mutation set (this is your ETL merge process).</li>
+<li>Append bucket ids to insertion records. A <code>BucketIdResolver</code>
+can help here.
+</li>
+<li>Group and sort your data appropriately.</li>
+<li>Issue mutation events to your coordinators.</li>
+<li>Close your coordinators.</li>
+<li>Abort or commit the transaction.</li>
+<li>Close your mutation client.</li>
+</ol>
+<p>
+See
+<code>ExampleUseCase</code>
+and
+<code>TestMutations.testUpdatesAndDeletes()</code>
+for some very simple usages.
+</p>
+
+</body>
+
+</html>
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java
new file mode 100644
index 0000000..656324c
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdException.java
@@ -0,0 +1,11 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+public class BucketIdException extends WorkerException {
+
+ private static final long serialVersionUID = 1L;
+
+ BucketIdException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java
new file mode 100644
index 0000000..dab2072
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolver.java
@@ -0,0 +1,11 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+/** Computes and appends bucket ids to records that are due to be inserted. */
+public interface BucketIdResolver {
+
+ Object attachBucketIdToRecord(Object record);
+
+ /** See: {@link org.apache.hadoop.hive.ql.exec.ReduceSinkOperator#computeBucketNumber(Object, int)}. */
+ int computeBucketId(Object record);
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java
new file mode 100644
index 0000000..dbed9e1
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/BucketIdResolverImpl.java
@@ -0,0 +1,76 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+
+/**
+ * Implementation of a {@link BucketIdResolver} that includes the logic required to calculate a bucket id from a record
+ * that is consistent with Hive's own internal computation scheme.
+ */
+public class BucketIdResolverImpl implements BucketIdResolver {
+
+ private static final long INVALID_TRANSACTION_ID = -1L;
+ private static final long INVALID_ROW_ID = -1L;
+
+ private final SettableStructObjectInspector structObjectInspector;
+ private final StructField[] bucketFields;
+ private final int totalBuckets;
+ private final StructField recordIdentifierField;
+
+ /**
+ * Note that all column indexes are with respect to your record structure, not the Hive table structure. Bucket column
+ * indexes must be presented in the same order as they are in the Hive table definition.
+ */
+ public BucketIdResolverImpl(ObjectInspector objectInspector, int recordIdColumn, int totalBuckets, int[] bucketColumns) {
+ this.totalBuckets = totalBuckets;
+ if (!(objectInspector instanceof SettableStructObjectInspector)) {
+ throw new IllegalArgumentException("Serious problem, expected a StructObjectInspector, " + "but got a "
+ + objectInspector.getClass().getName());
+ }
+
+ if (bucketColumns.length < 1) {
+ throw new IllegalArgumentException("No bucket column indexes set.");
+ }
+ structObjectInspector = (SettableStructObjectInspector) objectInspector;
+ List<? extends StructField> structFields = structObjectInspector.getAllStructFieldRefs();
+
+ recordIdentifierField = structFields.get(recordIdColumn);
+
+ bucketFields = new StructField[bucketColumns.length];
+ for (int i = 0; i < bucketColumns.length; i++) {
+ int bucketColumnsIndex = bucketColumns[i];
+ bucketFields[i] = structFields.get(bucketColumnsIndex);
+ }
+ }
+
+ @Override
+ public Object attachBucketIdToRecord(Object record) {
+ int bucketId = computeBucketId(record);
+ RecordIdentifier recordIdentifier = new RecordIdentifier(INVALID_TRANSACTION_ID, bucketId, INVALID_ROW_ID);
+ structObjectInspector.setStructFieldData(record, recordIdentifierField, recordIdentifier);
+ return record;
+ }
+
+ /** Based on: {@link org.apache.hadoop.hive.ql.exec.ReduceSinkOperator#computeBucketNumber(Object, int)}. */
+ @Override
+ public int computeBucketId(Object record) {
+ int bucketId = 1;
+
+ for (int columnIndex = 0; columnIndex < bucketFields.length; columnIndex++) {
+ Object columnValue = structObjectInspector.getStructFieldData(record, bucketFields[columnIndex]);
+ bucketId = bucketId * 31 + ObjectInspectorUtils.hashCode(columnValue, bucketFields[columnIndex].getFieldObjectInspector());
+ }
+
+ if (bucketId < 0) {
+ bucketId = -1 * bucketId;
+ }
+
+ return bucketId % totalBuckets;
+ }
+
+}
[25/25] hive git commit: HIVE-11162 : LLAP: merge master into branch
(Sergey Shelukhin)
Posted by se...@apache.org.
HIVE-11162 : LLAP: merge master into branch (Sergey Shelukhin)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/25522011
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/25522011
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/25522011
Branch: refs/heads/llap
Commit: 25522011976d6cb92e44299a281f7e88eacab219
Parents: 4339fd8 023c06c
Author: Sergey Shelukhin <se...@apache.org>
Authored: Wed Jul 1 18:41:44 2015 -0700
Committer: Sergey Shelukhin <se...@apache.org>
Committed: Wed Jul 1 18:41:44 2015 -0700
----------------------------------------------------------------------
.gitignore | 1 +
.../java/org/apache/hive/beeline/Commands.java | 22 +-
bin/ext/hplsql.sh | 37 +
bin/hplsql | 25 +
bin/hplsql.cmd | 58 +
data/files/encoding-utf8.txt | 12 +
.../src/test/e2e/templeton/deployers/env.sh | 5 +-
hcatalog/streaming/pom.xml | 6 +
.../streaming/mutate/HiveConfFactory.java | 63 +
.../mutate/UgiMetaStoreClientFactory.java | 102 +
.../streaming/mutate/client/AcidTable.java | 112 +
.../mutate/client/AcidTableSerializer.java | 100 +
.../mutate/client/ClientException.java | 15 +
.../mutate/client/ConnectionException.java | 15 +
.../streaming/mutate/client/MutatorClient.java | 140 +
.../mutate/client/MutatorClientBuilder.java | 115 +
.../streaming/mutate/client/TableType.java | 37 +
.../streaming/mutate/client/Transaction.java | 114 +
.../mutate/client/TransactionException.java | 15 +
.../mutate/client/lock/HeartbeatFactory.java | 30 +
.../mutate/client/lock/HeartbeatTimerTask.java | 66 +
.../streaming/mutate/client/lock/Lock.java | 282 ++
.../mutate/client/lock/LockException.java | 15 +
.../mutate/client/lock/LockFailureListener.java | 26 +
.../mutate/doc-files/system-overview.dot | 27 +
.../hive/hcatalog/streaming/mutate/package.html | 495 +++
.../mutate/worker/BucketIdException.java | 11 +
.../mutate/worker/BucketIdResolver.java | 11 +
.../mutate/worker/BucketIdResolverImpl.java | 76 +
.../mutate/worker/CreatePartitionHelper.java | 83 +
.../mutate/worker/GroupRevisitedException.java | 11 +
.../mutate/worker/GroupingValidator.java | 74 +
.../streaming/mutate/worker/Mutator.java | 21 +
.../mutate/worker/MutatorCoordinator.java | 281 ++
.../worker/MutatorCoordinatorBuilder.java | 76 +
.../streaming/mutate/worker/MutatorFactory.java | 16 +
.../streaming/mutate/worker/MutatorImpl.java | 84 +
.../streaming/mutate/worker/OperationType.java | 7 +
.../worker/PartitionCreationException.java | 15 +
.../mutate/worker/RecordInspector.java | 11 +
.../mutate/worker/RecordInspectorImpl.java | 45 +
.../mutate/worker/RecordSequenceException.java | 11 +
.../mutate/worker/SequenceValidator.java | 49 +
.../mutate/worker/WorkerException.java | 15 +
.../streaming/mutate/ExampleUseCase.java | 82 +
.../streaming/mutate/MutableRecord.java | 50 +
.../mutate/ReflectiveMutatorFactory.java | 51 +
.../streaming/mutate/StreamingAssert.java | 191 +
.../streaming/mutate/StreamingTestUtils.java | 261 ++
.../streaming/mutate/TestMutations.java | 544 +++
.../mutate/client/TestAcidTableSerializer.java | 66 +
.../mutate/client/TestMutatorClient.java | 176 +
.../mutate/client/TestTransaction.java | 95 +
.../client/lock/TestHeartbeatTimerTask.java | 100 +
.../streaming/mutate/client/lock/TestLock.java | 283 ++
.../mutate/worker/TestBucketIdResolverImpl.java | 38 +
.../mutate/worker/TestGroupingValidator.java | 70 +
.../mutate/worker/TestMutatorCoordinator.java | 234 ++
.../mutate/worker/TestMutatorImpl.java | 99 +
.../mutate/worker/TestRecordInspectorImpl.java | 31 +
.../mutate/worker/TestSequenceValidator.java | 91 +
hplsql/pom.xml | 123 +
.../antlr4/org/apache/hive/hplsql/Hplsql.g4 | 1426 ++++++++
.../java/org/apache/hive/hplsql/Arguments.java | 206 ++
.../main/java/org/apache/hive/hplsql/Conf.java | 175 +
.../main/java/org/apache/hive/hplsql/Conn.java | 243 ++
.../java/org/apache/hive/hplsql/Converter.java | 56 +
.../main/java/org/apache/hive/hplsql/Copy.java | 426 +++
.../main/java/org/apache/hive/hplsql/Exec.java | 1950 +++++++++++
.../java/org/apache/hive/hplsql/Expression.java | 574 +++
.../main/java/org/apache/hive/hplsql/File.java | 132 +
.../java/org/apache/hive/hplsql/Handler.java | 41 +
.../java/org/apache/hive/hplsql/Hplsql.java | 25 +
.../java/org/apache/hive/hplsql/Interval.java | 109 +
.../main/java/org/apache/hive/hplsql/Query.java | 155 +
.../main/java/org/apache/hive/hplsql/Scope.java | 69 +
.../java/org/apache/hive/hplsql/Select.java | 411 +++
.../java/org/apache/hive/hplsql/Signal.java | 48 +
.../main/java/org/apache/hive/hplsql/Stmt.java | 1021 ++++++
.../org/apache/hive/hplsql/StreamGobbler.java | 51 +
.../main/java/org/apache/hive/hplsql/Timer.java | 59 +
.../main/java/org/apache/hive/hplsql/Udf.java | 117 +
.../main/java/org/apache/hive/hplsql/Utils.java | 289 ++
.../main/java/org/apache/hive/hplsql/Var.java | 430 +++
.../apache/hive/hplsql/functions/Function.java | 709 ++++
.../hive/hplsql/functions/FunctionDatetime.java | 151 +
.../hive/hplsql/functions/FunctionMisc.java | 188 +
.../hive/hplsql/functions/FunctionOra.java | 231 ++
.../hive/hplsql/functions/FunctionString.java | 276 ++
.../hive/beeline/TestBeeLineWithArgs.java | 87 +
.../hadoop/hive/metastore/ObjectStore.java | 26 +-
pom.xml | 1 +
.../org/apache/hadoop/hive/ql/ErrorMsg.java | 9 +-
.../ql/io/orc/RunLengthIntegerReaderV2.java | 4 +-
.../hadoop/hive/ql/metadata/HiveUtils.java | 8 -
.../HiveJoinPushTransitivePredicatesRule.java | 139 +
.../calcite/translator/ASTConverter.java | 20 +-
.../translator/PlanModifierForASTConv.java | 12 +-
.../hadoop/hive/ql/parse/CalcitePlanner.java | 5 +-
.../ql/udf/generic/GenericUDAFComputeStats.java | 4 +-
.../generic/GenericUDAFHistogramNumeric.java | 2 +-
.../generic/GenericUDAFPercentileApprox.java | 2 +-
.../udf/generic/NumDistinctValueEstimator.java | 18 +
.../hive/ql/udf/generic/NumericHistogram.java | 14 +
.../hadoop/hive/ql/util/JavaDataModel.java | 30 -
.../insert_non_utf8_encoding_table.q | 20 +
.../results/clientpositive/auto_join13.q.out | 26 +-
.../auto_join_without_localtask.q.out | 218 +-
.../clientpositive/cbo_rp_auto_join1.q.out | 57 +-
.../clientpositive/correlationoptimizer6.q.out | 163 +-
.../insert_non_utf8_encoding_table.q.out | 89 +
ql/src/test/results/clientpositive/join13.q.out | 32 +-
ql/src/test/results/clientpositive/join32.q.out | 36 +-
.../clientpositive/join32_lessSize.q.out | 118 +-
ql/src/test/results/clientpositive/join33.q.out | 36 +-
.../clientpositive/join_alt_syntax.q.out | 104 +-
.../clientpositive/join_cond_pushdown_1.q.out | 42 +-
.../clientpositive/join_cond_pushdown_2.q.out | 62 +-
.../clientpositive/join_cond_pushdown_3.q.out | 42 +-
.../clientpositive/join_cond_pushdown_4.q.out | 62 +-
.../clientpositive/mapjoin_mapjoin.q.out | 120 +-
.../clientpositive/spark/auto_join13.q.out | 40 +-
.../spark/auto_join_without_localtask.q.out | 90 +-
.../results/clientpositive/spark/join13.q.out | 82 +-
.../results/clientpositive/spark/join32.q.out | 102 +-
.../clientpositive/spark/join32_lessSize.q.out | 232 +-
.../results/clientpositive/spark/join33.q.out | 102 +-
.../clientpositive/spark/join_alt_syntax.q.out | 272 +-
.../spark/join_cond_pushdown_1.q.out | 70 +-
.../spark/join_cond_pushdown_2.q.out | 134 +-
.../spark/join_cond_pushdown_3.q.out | 70 +-
.../spark/join_cond_pushdown_4.q.out | 134 +-
.../clientpositive/spark/mapjoin_mapjoin.q.out | 202 +-
.../clientpositive/tez/explainuser_1.q.out | 363 +-
.../clientpositive/tez/explainuser_2.q.out | 3258 +++++++++---------
.../clientpositive/tez/mapjoin_mapjoin.q.out | 266 +-
.../apache/hadoop/hive/serde2/SerDeUtils.java | 2 +-
.../cli/session/HiveSessionImplwithUGI.java | 14 +-
.../apache/hive/service/server/HiveServer2.java | 6 +-
139 files changed, 18599 insertions(+), 3433 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/25522011/pom.xml
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/25522011/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RunLengthIntegerReaderV2.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/25522011/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/25522011/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
----------------------------------------------------------------------
[14/25] hive git commit: HIVE-10794: Remove the dependence from
ErrorMsg to HiveUtils (Owen O'Malley via Prasanth Jayachandran)
Posted by se...@apache.org.
HIVE-10794: Remove the dependence from ErrorMsg to HiveUtils (Owen O'Malley via Prasanth Jayachandran)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/f755652f
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/f755652f
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/f755652f
Branch: refs/heads/llap
Commit: f755652ff053c57373a4c072d4c93cf9b7169e01
Parents: 052643c
Author: Prasanth Jayachandran <j....@gmail.com>
Authored: Wed Jul 1 10:00:08 2015 -0700
Committer: Prasanth Jayachandran <j....@gmail.com>
Committed: Wed Jul 1 10:00:08 2015 -0700
----------------------------------------------------------------------
ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java | 6 ++++--
.../java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java | 8 --------
2 files changed, 4 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/f755652f/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index 20509ce..f012b72 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -665,6 +665,8 @@ public enum ErrorMsg {
return sb.toString();
}
+ static final String LINE_SEP = System.getProperty("line.separator");
+
public static void renderOrigin(StringBuilder sb, ASTNodeOrigin origin) {
while (origin != null) {
sb.append(" in definition of ");
@@ -672,9 +674,9 @@ public enum ErrorMsg {
sb.append(" ");
sb.append(origin.getObjectName());
sb.append(" [");
- sb.append(HiveUtils.LINE_SEP);
+ sb.append(LINE_SEP);
sb.append(origin.getObjectDefinition());
- sb.append(HiveUtils.LINE_SEP);
+ sb.append(LINE_SEP);
sb.append("] used as ");
sb.append(origin.getUsageAlias());
sb.append(" at ");
http://git-wip-us.apache.org/repos/asf/hive/blob/f755652f/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
index c4633f6..719728d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
@@ -45,14 +45,6 @@ import org.apache.hadoop.util.ReflectionUtils;
*/
public final class HiveUtils {
- public static final char QUOTE = '"';
- public static final char COLON = ':';
- public static final String LBRACKET = "[";
- public static final String RBRACKET = "]";
- public static final String LBRACE = "{";
- public static final String RBRACE = "}";
- public static final String LINE_SEP = System.getProperty("line.separator");
-
public static String escapeString(String str) {
int length = str.length();
StringBuilder escape = new StringBuilder(length + 16);
[08/25] hive git commit: HIVE-11140 auto compute PROJ_HOME in
hcatalog/src/test/e2e/templeton/deployers/env.sh (Eugene Koifman,
reviewed by Thejas Nair)
Posted by se...@apache.org.
HIVE-11140 auto compute PROJ_HOME in hcatalog/src/test/e2e/templeton/deployers/env.sh (Eugene Koifman, reviewed by Thejas Nair)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ffce2258
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ffce2258
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ffce2258
Branch: refs/heads/llap
Commit: ffce225896d73ac0af1af4afb9d5bef53699ab37
Parents: 994d98c
Author: Eugene Koifman <ek...@hortonworks.com>
Authored: Tue Jun 30 15:11:43 2015 -0700
Committer: Eugene Koifman <ek...@hortonworks.com>
Committed: Tue Jun 30 15:11:43 2015 -0700
----------------------------------------------------------------------
hcatalog/src/test/e2e/templeton/deployers/env.sh | 5 ++---
1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/ffce2258/hcatalog/src/test/e2e/templeton/deployers/env.sh
----------------------------------------------------------------------
diff --git a/hcatalog/src/test/e2e/templeton/deployers/env.sh b/hcatalog/src/test/e2e/templeton/deployers/env.sh
index 8b719f2..804bdd4 100755
--- a/hcatalog/src/test/e2e/templeton/deployers/env.sh
+++ b/hcatalog/src/test/e2e/templeton/deployers/env.sh
@@ -42,9 +42,8 @@ if [ -z ${TEZ_VERSION} ]; then
fi
#Root of project source tree
-if [ -z ${PROJ_HOME} ]; then
- export PROJ_HOME=/Users/${USER}/dev/hive
-fi
+current_dir=$( cd "$( dirname "${BASH_SOURCE[0]}" )"/../../../../.. && pwd )
+export PROJ_HOME=`dirname $current_dir`
export HIVE_HOME=${PROJ_HOME}/packaging/target/apache-hive-${HIVE_VERSION}-bin/apache-hive-${HIVE_VERSION}-bin
if [ -z ${HADOOP_HOME} ]; then
[06/25] hive git commit: HIVE-10165 Improve hive-hcatalog-streaming
extensibility and support updates and deletes (Eliot West via gates)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java
new file mode 100644
index 0000000..9aab346
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/CreatePartitionHelper.java
@@ -0,0 +1,83 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Utility class that can create new table partitions within the {@link IMetaStoreClient meta store}. */
+class CreatePartitionHelper {
+
+ private static final Logger LOG = LoggerFactory.getLogger(CreatePartitionHelper.class);
+
+ private final IMetaStoreClient metaStoreClient;
+ private final String databaseName;
+ private final String tableName;
+
+ CreatePartitionHelper(IMetaStoreClient metaStoreClient, String databaseName, String tableName) {
+ this.metaStoreClient = metaStoreClient;
+ this.databaseName = databaseName;
+ this.tableName = tableName;
+ }
+
+ /** Returns the expected {@link Path} for a given partition value. */
+ Path getPathForPartition(List<String> newPartitionValues) throws WorkerException {
+ try {
+ String location;
+ if (newPartitionValues.isEmpty()) {
+ location = metaStoreClient.getTable(databaseName, tableName).getSd().getLocation();
+ } else {
+ location = metaStoreClient.getPartition(databaseName, tableName, newPartitionValues).getSd().getLocation();
+ }
+ LOG.debug("Found path {} for partition {}", location, newPartitionValues);
+ return new Path(location);
+ } catch (NoSuchObjectException e) {
+ throw new WorkerException("Table not found '" + databaseName + "." + tableName + "'.", e);
+ } catch (TException e) {
+ throw new WorkerException("Failed to get path for partitions '" + newPartitionValues + "' on table '"
+ + databaseName + "." + tableName + "' with meta store: " + metaStoreClient, e);
+ }
+ }
+
+ /** Creates the specified partition if it does not already exist. Does nothing if the table is unpartitioned. */
+ void createPartitionIfNotExists(List<String> newPartitionValues) throws WorkerException {
+ if (newPartitionValues.isEmpty()) {
+ return;
+ }
+
+ try {
+ LOG.debug("Attempting to create partition (if not exists) {}.{}:{}", databaseName, tableName, newPartitionValues);
+ Table table = metaStoreClient.getTable(databaseName, tableName);
+
+ Partition partition = new Partition();
+ partition.setDbName(table.getDbName());
+ partition.setTableName(table.getTableName());
+ StorageDescriptor partitionSd = new StorageDescriptor(table.getSd());
+ partitionSd.setLocation(table.getSd().getLocation() + Path.SEPARATOR
+ + Warehouse.makePartName(table.getPartitionKeys(), newPartitionValues));
+ partition.setSd(partitionSd);
+ partition.setValues(newPartitionValues);
+
+ metaStoreClient.add_partition(partition);
+ } catch (AlreadyExistsException e) {
+ LOG.debug("Partition already exisits: {}.{}:{}", databaseName, tableName, newPartitionValues);
+ } catch (NoSuchObjectException e) {
+ LOG.error("Failed to create partition : " + newPartitionValues, e);
+ throw new PartitionCreationException("Table not found '" + databaseName + "." + tableName + "'.", e);
+ } catch (TException e) {
+ LOG.error("Failed to create partition : " + newPartitionValues, e);
+ throw new PartitionCreationException("Failed to create partition '" + newPartitionValues + "' on table '"
+ + databaseName + "." + tableName + "'", e);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java
new file mode 100644
index 0000000..f8e46d6
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupRevisitedException.java
@@ -0,0 +1,11 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+public class GroupRevisitedException extends WorkerException {
+
+ private static final long serialVersionUID = 1L;
+
+ GroupRevisitedException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java
new file mode 100644
index 0000000..8ae3904
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/GroupingValidator.java
@@ -0,0 +1,74 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * Tracks the (partition, bucket) combinations that have been encountered, checking that a group is not revisited.
+ * Potentially memory intensive.
+ */
+class GroupingValidator {
+
+ private final Map<String, Set<Integer>> visited;
+ private final StringBuffer partitionKeyBuilder;
+ private long groups;
+ private String lastPartitionKey;
+ private int lastBucketId = -1;
+
+ GroupingValidator() {
+ visited = new HashMap<String, Set<Integer>>();
+ partitionKeyBuilder = new StringBuffer(64);
+ }
+
+ /**
+ * Checks that this group is either the same as the last or is a new group.
+ */
+ boolean isInSequence(List<String> partitionValues, int bucketId) {
+ String partitionKey = getPartitionKey(partitionValues);
+ if (Objects.equals(lastPartitionKey, partitionKey) && lastBucketId == bucketId) {
+ return true;
+ }
+ lastPartitionKey = partitionKey;
+ lastBucketId = bucketId;
+
+ Set<Integer> bucketIdSet = visited.get(partitionKey);
+ if (bucketIdSet == null) {
+ // If the bucket id set component of this data structure proves to be too large there is the
+ // option of moving it to Trove or HPPC in an effort to reduce size.
+ bucketIdSet = new HashSet<>();
+ visited.put(partitionKey, bucketIdSet);
+ }
+
+ boolean newGroup = bucketIdSet.add(bucketId);
+ if (newGroup) {
+ groups++;
+ }
+ return newGroup;
+ }
+
+ private String getPartitionKey(List<String> partitionValues) {
+ partitionKeyBuilder.setLength(0);
+ boolean first = true;
+ for (String element : partitionValues) {
+ if (first) {
+ first = false;
+ } else {
+ partitionKeyBuilder.append('/');
+ }
+ partitionKeyBuilder.append(element);
+ }
+ String partitionKey = partitionKeyBuilder.toString();
+ return partitionKey;
+ }
+
+ @Override
+ public String toString() {
+ return "GroupingValidator [groups=" + groups + ",lastPartitionKey=" + lastPartitionKey + ",lastBucketId="
+ + lastBucketId + "]";
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java
new file mode 100644
index 0000000..96ecce9
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/Mutator.java
@@ -0,0 +1,21 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.Closeable;
+import java.io.Flushable;
+import java.io.IOException;
+
+/**
+ * Interface for submitting mutation events to a given partition and bucket in an ACID table. Requires records to arrive
+ * in the order defined by the {@link SequenceValidator}.
+ */
+public interface Mutator extends Closeable, Flushable {
+
+ void insert(Object record) throws IOException;
+
+ void update(Object record) throws IOException;
+
+ void delete(Object record) throws IOException;
+
+ void flush() throws IOException;
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
new file mode 100644
index 0000000..96f05e5
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinator.java
@@ -0,0 +1,281 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.Closeable;
+import java.io.Flushable;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.JavaUtils;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.hive.ql.io.RecordUpdater;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Orchestrates the application of an ordered sequence of mutation events to a given ACID table. Events must be grouped
+ * by partition, then bucket and ordered by origTxnId, then rowId. Ordering is enforced by the {@link SequenceValidator}
+ * and grouping is by the {@link GroupingValidator}. An acid delta file is created for each combination partition, and
+ * bucket id (a single transaction id is implied). Once a delta file has been closed it cannot be reopened. Therefore
+ * care is needed as to group the data correctly otherwise failures will occur if a delta belonging to group has been
+ * previously closed. The {@link MutatorCoordinator} will seamlessly handle transitions between groups, creating and
+ * closing {@link Mutator Mutators} as needed to write to the appropriate partition and bucket. New partitions will be
+ * created in the meta store if {@link AcidTable#createPartitions()} is set.
+ * <p/>
+ * {@link #insert(List, Object) Insert} events must be artificially assigned appropriate bucket ids in the preceding
+ * grouping phase so that they are grouped correctly. Note that any transaction id or row id assigned to the
+ * {@link RecordIdentifier RecordIdentifier} of such events will be ignored by both the coordinator and the underlying
+ * {@link RecordUpdater}.
+ */
+public class MutatorCoordinator implements Closeable, Flushable {
+
+ private static final Logger LOG = LoggerFactory.getLogger(MutatorCoordinator.class);
+
+ private final IMetaStoreClient metaStoreClient;
+ private final MutatorFactory mutatorFactory;
+ private final GroupingValidator groupingValidator;
+ private final SequenceValidator sequenceValidator;
+ private final AcidTable table;
+ private final RecordInspector recordInspector;
+ private final CreatePartitionHelper partitionHelper;
+ private final AcidOutputFormat<?, ?> outputFormat;
+ private final BucketIdResolver bucketIdResolver;
+ private final HiveConf configuration;
+ private final boolean deleteDeltaIfExists;
+
+ private int bucketId;
+ private List<String> partitionValues;
+ private Path partitionPath;
+ private Mutator mutator;
+
+ MutatorCoordinator(IMetaStoreClient metaStoreClient, HiveConf configuration, MutatorFactory mutatorFactory,
+ AcidTable table, boolean deleteDeltaIfExists) throws WorkerException {
+ this(metaStoreClient, configuration, mutatorFactory, new CreatePartitionHelper(metaStoreClient,
+ table.getDatabaseName(), table.getTableName()), new GroupingValidator(), new SequenceValidator(), table,
+ deleteDeltaIfExists);
+ }
+
+ /** Visible for testing only. */
+ MutatorCoordinator(IMetaStoreClient metaStoreClient, HiveConf configuration, MutatorFactory mutatorFactory,
+ CreatePartitionHelper partitionHelper, GroupingValidator groupingValidator, SequenceValidator sequenceValidator,
+ AcidTable table, boolean deleteDeltaIfExists) throws WorkerException {
+ this.metaStoreClient = metaStoreClient;
+ this.configuration = configuration;
+ this.mutatorFactory = mutatorFactory;
+ this.partitionHelper = partitionHelper;
+ this.groupingValidator = groupingValidator;
+ this.sequenceValidator = sequenceValidator;
+ this.table = table;
+ this.deleteDeltaIfExists = deleteDeltaIfExists;
+ this.recordInspector = this.mutatorFactory.newRecordInspector();
+ bucketIdResolver = this.mutatorFactory.newBucketIdResolver(table.getTotalBuckets());
+
+ bucketId = -1;
+ outputFormat = createOutputFormat(table.getOutputFormatName(), configuration);
+ }
+
+ /**
+ * We expect records grouped by (partitionValues,bucketId) and ordered by (origTxnId,rowId).
+ *
+ * @throws BucketIdException The bucket ID in the {@link RecordIdentifier} of the record does not match that computed
+ * using the values in the record's bucketed columns.
+ * @throws RecordSequenceException The record was submitted that was not in the correct ascending (origTxnId, rowId)
+ * sequence.
+ * @throws GroupRevisitedException If an event was submitted for a (partition, bucketId) combination that has already
+ * been closed.
+ * @throws PartitionCreationException Could not create a new partition in the meta store.
+ * @throws WorkerException
+ */
+ public void insert(List<String> partitionValues, Object record) throws WorkerException {
+ reconfigureState(OperationType.INSERT, partitionValues, record);
+ try {
+ mutator.insert(record);
+ LOG.debug("Inserted into partition={}, record={}", partitionValues, record);
+ } catch (IOException e) {
+ throw new WorkerException("Failed to insert record '" + record + " using mutator '" + mutator + "'.", e);
+ }
+ }
+
+ /**
+ * We expect records grouped by (partitionValues,bucketId) and ordered by (origTxnId,rowId).
+ *
+ * @throws BucketIdException The bucket ID in the {@link RecordIdentifier} of the record does not match that computed
+ * using the values in the record's bucketed columns.
+ * @throws RecordSequenceException The record was submitted that was not in the correct ascending (origTxnId, rowId)
+ * sequence.
+ * @throws GroupRevisitedException If an event was submitted for a (partition, bucketId) combination that has already
+ * been closed.
+ * @throws PartitionCreationException Could not create a new partition in the meta store.
+ * @throws WorkerException
+ */
+ public void update(List<String> partitionValues, Object record) throws WorkerException {
+ reconfigureState(OperationType.UPDATE, partitionValues, record);
+ try {
+ mutator.update(record);
+ LOG.debug("Updated in partition={}, record={}", partitionValues, record);
+ } catch (IOException e) {
+ throw new WorkerException("Failed to update record '" + record + " using mutator '" + mutator + "'.", e);
+ }
+ }
+
+ /**
+ * We expect records grouped by (partitionValues,bucketId) and ordered by (origTxnId,rowId).
+ *
+ * @throws BucketIdException The bucket ID in the {@link RecordIdentifier} of the record does not match that computed
+ * using the values in the record's bucketed columns.
+ * @throws RecordSequenceException The record was submitted that was not in the correct ascending (origTxnId, rowId)
+ * sequence.
+ * @throws GroupRevisitedException If an event was submitted for a (partition, bucketId) combination that has already
+ * been closed.
+ * @throws PartitionCreationException Could not create a new partition in the meta store.
+ * @throws WorkerException
+ */
+ public void delete(List<String> partitionValues, Object record) throws WorkerException {
+ reconfigureState(OperationType.DELETE, partitionValues, record);
+ try {
+ mutator.delete(record);
+ LOG.debug("Deleted from partition={}, record={}", partitionValues, record);
+ } catch (IOException e) {
+ throw new WorkerException("Failed to delete record '" + record + " using mutator '" + mutator + "'.", e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ if (mutator != null) {
+ mutator.close();
+ }
+ } finally {
+ metaStoreClient.close();
+ }
+ }
+
+ @Override
+ public void flush() throws IOException {
+ if (mutator != null) {
+ mutator.flush();
+ }
+ }
+
+ private void reconfigureState(OperationType operationType, List<String> newPartitionValues, Object record)
+ throws WorkerException {
+ RecordIdentifier newRecordIdentifier = extractRecordIdentifier(operationType, newPartitionValues, record);
+ int newBucketId = newRecordIdentifier.getBucketId();
+
+ if (newPartitionValues == null) {
+ newPartitionValues = Collections.emptyList();
+ }
+
+ try {
+ if (partitionHasChanged(newPartitionValues)) {
+ if (table.createPartitions()) {
+ partitionHelper.createPartitionIfNotExists(newPartitionValues);
+ }
+ Path newPartitionPath = partitionHelper.getPathForPartition(newPartitionValues);
+ resetMutator(newBucketId, newPartitionValues, newPartitionPath);
+ } else if (bucketIdHasChanged(newBucketId)) {
+ resetMutator(newBucketId, partitionValues, partitionPath);
+ } else {
+ validateRecordSequence(operationType, newRecordIdentifier);
+ }
+ } catch (IOException e) {
+ throw new WorkerException("Failed to reset mutator when performing " + operationType + " of record: " + record, e);
+ }
+ }
+
+ private RecordIdentifier extractRecordIdentifier(OperationType operationType, List<String> newPartitionValues,
+ Object record) throws BucketIdException {
+ RecordIdentifier recordIdentifier = recordInspector.extractRecordIdentifier(record);
+ int computedBucketId = bucketIdResolver.computeBucketId(record);
+ if (operationType != OperationType.DELETE && recordIdentifier.getBucketId() != computedBucketId) {
+ throw new BucketIdException("RecordIdentifier.bucketId != computed bucketId (" + computedBucketId
+ + ") for record " + recordIdentifier + " in partition " + newPartitionValues + ".");
+ }
+ return recordIdentifier;
+ }
+
+ private void resetMutator(int newBucketId, List<String> newPartitionValues, Path newPartitionPath)
+ throws IOException, GroupRevisitedException {
+ if (mutator != null) {
+ mutator.close();
+ }
+ validateGrouping(newPartitionValues, newBucketId);
+ sequenceValidator.reset();
+ if (deleteDeltaIfExists) {
+ // TODO: Should this be the concern of the mutator?
+ deleteDeltaIfExists(newPartitionPath, table.getTransactionId(), newBucketId);
+ }
+ mutator = mutatorFactory.newMutator(outputFormat, table.getTransactionId(), newPartitionPath, newBucketId);
+ bucketId = newBucketId;
+ partitionValues = newPartitionValues;
+ partitionPath = newPartitionPath;
+ LOG.debug("Reset mutator: bucketId={}, partition={}, partitionPath={}", bucketId, partitionValues, partitionPath);
+ }
+
+ private boolean partitionHasChanged(List<String> newPartitionValues) {
+ boolean partitionHasChanged = !Objects.equals(this.partitionValues, newPartitionValues);
+ if (partitionHasChanged) {
+ LOG.debug("Partition changed from={}, to={}", this.partitionValues, newPartitionValues);
+ }
+ return partitionHasChanged;
+ }
+
+ private boolean bucketIdHasChanged(int newBucketId) {
+ boolean bucketIdHasChanged = this.bucketId != newBucketId;
+ if (bucketIdHasChanged) {
+ LOG.debug("Bucket ID changed from={}, to={}", this.bucketId, newBucketId);
+ }
+ return bucketIdHasChanged;
+ }
+
+ private void validateGrouping(List<String> newPartitionValues, int newBucketId) throws GroupRevisitedException {
+ if (!groupingValidator.isInSequence(newPartitionValues, bucketId)) {
+ throw new GroupRevisitedException("Group out of sequence: state=" + groupingValidator + ", partition="
+ + newPartitionValues + ", bucketId=" + newBucketId);
+ }
+ }
+
+ private void validateRecordSequence(OperationType operationType, RecordIdentifier newRecordIdentifier)
+ throws RecordSequenceException {
+ boolean identiferOutOfSequence = operationType != OperationType.INSERT
+ && !sequenceValidator.isInSequence(newRecordIdentifier);
+ if (identiferOutOfSequence) {
+ throw new RecordSequenceException("Records not in sequence: state=" + sequenceValidator + ", recordIdentifier="
+ + newRecordIdentifier);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ private AcidOutputFormat<?, ?> createOutputFormat(String outputFormatName, HiveConf configuration)
+ throws WorkerException {
+ try {
+ return (AcidOutputFormat<?, ?>) ReflectionUtils.newInstance(JavaUtils.loadClass(outputFormatName), configuration);
+ } catch (ClassNotFoundException e) {
+ throw new WorkerException("Could not locate class for '" + outputFormatName + "'.", e);
+ }
+ }
+
+ private void deleteDeltaIfExists(Path partitionPath, long transactionId, int bucketId) throws IOException {
+ Path deltaPath = AcidUtils.createFilename(partitionPath,
+ new AcidOutputFormat.Options(configuration)
+ .bucket(bucketId)
+ .minimumTransactionId(transactionId)
+ .maximumTransactionId(transactionId));
+ FileSystem fileSystem = deltaPath.getFileSystem(configuration);
+ if (fileSystem.exists(deltaPath)) {
+ LOG.info("Deleting existing delta path: {}", deltaPath);
+ fileSystem.delete(deltaPath, false);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
new file mode 100644
index 0000000..8851ea6
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorCoordinatorBuilder.java
@@ -0,0 +1,76 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.hcatalog.common.HCatUtil;
+import org.apache.hive.hcatalog.streaming.mutate.HiveConfFactory;
+import org.apache.hive.hcatalog.streaming.mutate.UgiMetaStoreClientFactory;
+import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable;
+
+/** Convenience class for building {@link MutatorCoordinator} instances. */
+public class MutatorCoordinatorBuilder {
+
+ private HiveConf configuration;
+ private MutatorFactory mutatorFactory;
+ private UserGroupInformation authenticatedUser;
+ private String metaStoreUri;
+ private AcidTable table;
+ private boolean deleteDeltaIfExists;
+
+ public MutatorCoordinatorBuilder configuration(HiveConf configuration) {
+ this.configuration = configuration;
+ return this;
+ }
+
+ public MutatorCoordinatorBuilder authenticatedUser(UserGroupInformation authenticatedUser) {
+ this.authenticatedUser = authenticatedUser;
+ return this;
+ }
+
+ public MutatorCoordinatorBuilder metaStoreUri(String metaStoreUri) {
+ this.metaStoreUri = metaStoreUri;
+ return this;
+ }
+
+ /** Set the destination ACID table for this client. */
+ public MutatorCoordinatorBuilder table(AcidTable table) {
+ this.table = table;
+ return this;
+ }
+
+ /**
+ * If the delta file already exists, delete it. THis is useful in a MapReduce setting where a number of task retries
+ * will attempt to write the same delta file.
+ */
+ public MutatorCoordinatorBuilder deleteDeltaIfExists() {
+ this.deleteDeltaIfExists = true;
+ return this;
+ }
+
+ public MutatorCoordinatorBuilder mutatorFactory(MutatorFactory mutatorFactory) {
+ this.mutatorFactory = mutatorFactory;
+ return this;
+ }
+
+ public MutatorCoordinator build() throws WorkerException, MetaException {
+ String user = authenticatedUser == null ? System.getProperty("user.name") : authenticatedUser.getShortUserName();
+ boolean secureMode = authenticatedUser == null ? false : authenticatedUser.hasKerberosCredentials();
+
+ configuration = HiveConfFactory.newInstance(configuration, this.getClass(), metaStoreUri);
+
+ IMetaStoreClient metaStoreClient;
+ try {
+ metaStoreClient = new UgiMetaStoreClientFactory(metaStoreUri, configuration, authenticatedUser, user, secureMode)
+ .newInstance(HCatUtil.getHiveMetastoreClient(configuration));
+ } catch (IOException e) {
+ throw new WorkerException("Could not create meta store client.", e);
+ }
+
+ return new MutatorCoordinator(metaStoreClient, configuration, mutatorFactory, table, deleteDeltaIfExists);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java
new file mode 100644
index 0000000..850054f
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorFactory.java
@@ -0,0 +1,16 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+
+public interface MutatorFactory {
+
+ Mutator newMutator(AcidOutputFormat<?, ?> outputFormat, long transactionId, Path partitionPath, int bucketId) throws IOException;
+
+ RecordInspector newRecordInspector();
+
+ BucketIdResolver newBucketIdResolver(int totalBuckets);
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java
new file mode 100644
index 0000000..0fe41d5
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/MutatorImpl.java
@@ -0,0 +1,84 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.ql.io.RecordUpdater;
+import org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+
+/** Base {@link Mutator} implementation. Creates a suitable {@link RecordUpdater} and delegates mutation events. */
+public class MutatorImpl implements Mutator {
+
+ private final long transactionId;
+ private final Path partitionPath;
+ private final int bucketId;
+ private final Configuration configuration;
+ private final int recordIdColumn;
+ private final ObjectInspector objectInspector;
+ private RecordUpdater updater;
+
+ public MutatorImpl(Configuration configuration, int recordIdColumn, ObjectInspector objectInspector,
+ AcidOutputFormat<?, ?> outputFormat, long transactionId, Path partitionPath, int bucketId) throws IOException {
+ this.configuration = configuration;
+ this.recordIdColumn = recordIdColumn;
+ this.objectInspector = objectInspector;
+ this.transactionId = transactionId;
+ this.partitionPath = partitionPath;
+ this.bucketId = bucketId;
+
+ updater = createRecordUpdater(outputFormat);
+ }
+
+ @Override
+ public void insert(Object record) throws IOException {
+ updater.insert(transactionId, record);
+ }
+
+ @Override
+ public void update(Object record) throws IOException {
+ updater.update(transactionId, record);
+ }
+
+ @Override
+ public void delete(Object record) throws IOException {
+ updater.delete(transactionId, record);
+ }
+
+ /**
+ * This implementation does intentionally nothing at this time. We only use a single transaction and
+ * {@link OrcRecordUpdater#flush()} will purposefully throw and exception in this instance. We keep this here in the
+ * event that we support multiple transactions and to make it clear that the omission of an invocation of
+ * {@link OrcRecordUpdater#flush()} was not a mistake.
+ */
+ @Override
+ public void flush() throws IOException {
+ // Intentionally do nothing
+ }
+
+ @Override
+ public void close() throws IOException {
+ updater.close(false);
+ updater = null;
+ }
+
+ @Override
+ public String toString() {
+ return "ObjectInspectorMutator [transactionId=" + transactionId + ", partitionPath=" + partitionPath
+ + ", bucketId=" + bucketId + "]";
+ }
+
+ protected RecordUpdater createRecordUpdater(AcidOutputFormat<?, ?> outputFormat) throws IOException {
+ return outputFormat.getRecordUpdater(
+ partitionPath,
+ new AcidOutputFormat.Options(configuration)
+ .inspector(objectInspector)
+ .bucket(bucketId)
+ .minimumTransactionId(transactionId)
+ .maximumTransactionId(transactionId)
+ .recordIdColumn(recordIdColumn));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java
new file mode 100644
index 0000000..5ecb1bb
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/OperationType.java
@@ -0,0 +1,7 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+enum OperationType {
+ INSERT,
+ UPDATE,
+ DELETE;
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java
new file mode 100644
index 0000000..5b59e01
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/PartitionCreationException.java
@@ -0,0 +1,15 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+public class PartitionCreationException extends WorkerException {
+
+ private static final long serialVersionUID = 1L;
+
+ PartitionCreationException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ PartitionCreationException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java
new file mode 100644
index 0000000..11ef0dd
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspector.java
@@ -0,0 +1,11 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+
+/** Provide a means to extract {@link RecordIdentifier} from record objects. */
+public interface RecordInspector {
+
+ /** Get the {@link RecordIdentifier} from the record - to be used for updates and deletes only. */
+ RecordIdentifier extractRecordIdentifier(Object record);
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java
new file mode 100644
index 0000000..18ee458
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordInspectorImpl.java
@@ -0,0 +1,45 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+
+/**
+ * Standard {@link RecordInspector} implementation that uses the supplied {@link ObjectInspector} and
+ * {@link AcidOutputFormat.Options#recordIdColumn(int) record id column} to extract {@link RecordIdentifier
+ * RecordIdentifiers}, and calculate bucket ids from records.
+ */
+public class RecordInspectorImpl implements RecordInspector {
+
+ private final StructObjectInspector structObjectInspector;
+ private final StructField recordIdentifierField;
+
+ /**
+ * Note that all column indexes are with respect to your record structure, not the Hive table structure.
+ */
+ public RecordInspectorImpl(ObjectInspector objectInspector, int recordIdColumn) {
+ if (!(objectInspector instanceof StructObjectInspector)) {
+ throw new IllegalArgumentException("Serious problem, expected a StructObjectInspector, " + "but got a "
+ + objectInspector.getClass().getName());
+ }
+
+ structObjectInspector = (StructObjectInspector) objectInspector;
+ List<? extends StructField> structFields = structObjectInspector.getAllStructFieldRefs();
+ recordIdentifierField = structFields.get(recordIdColumn);
+ }
+
+ public RecordIdentifier extractRecordIdentifier(Object record) {
+ return (RecordIdentifier) structObjectInspector.getStructFieldData(record, recordIdentifierField);
+ }
+
+ @Override
+ public String toString() {
+ return "RecordInspectorImpl [structObjectInspector=" + structObjectInspector + ", recordIdentifierField="
+ + recordIdentifierField + "]";
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java
new file mode 100644
index 0000000..6b034f1
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/RecordSequenceException.java
@@ -0,0 +1,11 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+public class RecordSequenceException extends WorkerException {
+
+ private static final long serialVersionUID = 1L;
+
+ RecordSequenceException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java
new file mode 100644
index 0000000..bcff4d6
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/SequenceValidator.java
@@ -0,0 +1,49 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Verifies that the sequence of {@link RecordIdentifier RecordIdentifiers} are in a valid order for insertion into an
+ * ACID delta file in a given partition and bucket.
+ */
+class SequenceValidator {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SequenceValidator.class);
+
+ private Long lastTxId;
+ private Long lastRowId;
+
+ SequenceValidator() {
+ }
+
+ boolean isInSequence(RecordIdentifier recordIdentifier) {
+ if (lastTxId != null && recordIdentifier.getTransactionId() < lastTxId) {
+ LOG.debug("Non-sequential transaction ID. Expected >{}, recordIdentifier={}", lastTxId, recordIdentifier);
+ return false;
+ } else if (lastTxId != null && recordIdentifier.getTransactionId() == lastTxId && lastRowId != null
+ && recordIdentifier.getRowId() <= lastRowId) {
+ LOG.debug("Non-sequential row ID. Expected >{}, recordIdentifier={}", lastRowId, recordIdentifier);
+ return false;
+ }
+ lastTxId = recordIdentifier.getTransactionId();
+ lastRowId = recordIdentifier.getRowId();
+ return true;
+ }
+
+ /**
+ * Validator must be reset for each new partition and or bucket.
+ */
+ void reset() {
+ lastTxId = null;
+ lastRowId = null;
+ LOG.debug("reset");
+ }
+
+ @Override
+ public String toString() {
+ return "SequenceValidator [lastTxId=" + lastTxId + ", lastRowId=" + lastRowId + "]";
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java
new file mode 100644
index 0000000..1fa1998
--- /dev/null
+++ b/hcatalog/streaming/src/java/org/apache/hive/hcatalog/streaming/mutate/worker/WorkerException.java
@@ -0,0 +1,15 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+public class WorkerException extends Exception {
+
+ private static final long serialVersionUID = 1L;
+
+ WorkerException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ WorkerException(String message) {
+ super(message);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java
new file mode 100644
index 0000000..86d70d4
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ExampleUseCase.java
@@ -0,0 +1,82 @@
+package org.apache.hive.hcatalog.streaming.mutate;
+
+import java.util.List;
+
+import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClient;
+import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClientBuilder;
+import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable;
+import org.apache.hive.hcatalog.streaming.mutate.client.Transaction;
+import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolver;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinator;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinatorBuilder;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorFactory;
+
+public class ExampleUseCase {
+
+ private String metaStoreUri;
+ private String databaseName;
+ private String tableName;
+ private boolean createPartitions = true;
+ private List<String> partitionValues1, partitionValues2, partitionValues3;
+ private Object record1, record2, record3;
+ private MutatorFactory mutatorFactory;
+
+ /* This is an illustration, not a functioning example. */
+ public void example() throws Exception {
+ // CLIENT/TOOL END
+ //
+ // Singleton instance in the job client
+
+ // Create a client to manage our transaction
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(databaseName, tableName, createPartitions)
+ .metaStoreUri(metaStoreUri)
+ .build();
+
+ // Get the transaction
+ Transaction transaction = client.newTransaction();
+
+ // Get serializable details of the destination tables
+ List<AcidTable> tables = client.getTables();
+
+ transaction.begin();
+
+ // CLUSTER / WORKER END
+ //
+ // Job submitted to the cluster
+ //
+
+ BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(tables.get(0).getTotalBuckets());
+ record1 = bucketIdResolver.attachBucketIdToRecord(record1);
+
+ // --------------------------------------------------------------
+ // DATA SHOULD GET SORTED BY YOUR ETL/MERGE PROCESS HERE
+ //
+ // Group the data by (partitionValues, ROW__ID.bucketId)
+ // Order the groups by (ROW__ID.lastTransactionId, ROW__ID.rowId)
+ // --------------------------------------------------------------
+
+ // One of these runs at the output of each reducer
+ //
+ MutatorCoordinator coordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(tables.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ coordinator.insert(partitionValues1, record1);
+ coordinator.update(partitionValues2, record2);
+ coordinator.delete(partitionValues3, record3);
+
+ coordinator.close();
+
+ // CLIENT/TOOL END
+ //
+ // The tasks have completed, control is back at the tool
+
+ transaction.commit();
+
+ client.close();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java
new file mode 100644
index 0000000..0d87a31
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/MutableRecord.java
@@ -0,0 +1,50 @@
+/**
+ * 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.hive.hcatalog.streaming.mutate;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.io.Text;
+
+public class MutableRecord {
+
+ // Column 0
+ public final int id;
+ // Column 1
+ public final Text msg;
+ // Column 2
+ public RecordIdentifier rowId;
+
+ public MutableRecord(int id, String msg, RecordIdentifier rowId) {
+ this.id = id;
+ this.msg = new Text(msg);
+ this.rowId = rowId;
+ }
+
+ public MutableRecord(int id, String msg) {
+ this.id = id;
+ this.msg = new Text(msg);
+ rowId = null;
+ }
+
+ @Override
+ public String toString() {
+ return "MutableRecord [id=" + id + ", msg=" + msg + ", rowId=" + rowId + "]";
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java
new file mode 100644
index 0000000..2a851c8
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/ReflectiveMutatorFactory.java
@@ -0,0 +1,51 @@
+package org.apache.hive.hcatalog.streaming.mutate;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolver;
+import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolverImpl;
+import org.apache.hive.hcatalog.streaming.mutate.worker.Mutator;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorFactory;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorImpl;
+import org.apache.hive.hcatalog.streaming.mutate.worker.RecordInspector;
+import org.apache.hive.hcatalog.streaming.mutate.worker.RecordInspectorImpl;
+
+public class ReflectiveMutatorFactory implements MutatorFactory {
+
+ private final int recordIdColumn;
+ private final ObjectInspector objectInspector;
+ private final Configuration configuration;
+ private final int[] bucketColumnIndexes;
+
+ public ReflectiveMutatorFactory(Configuration configuration, Class<?> recordClass, int recordIdColumn,
+ int[] bucketColumnIndexes) {
+ this.configuration = configuration;
+ this.recordIdColumn = recordIdColumn;
+ this.bucketColumnIndexes = bucketColumnIndexes;
+ objectInspector = ObjectInspectorFactory.getReflectionObjectInspector(recordClass,
+ ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+ }
+
+ @Override
+ public Mutator newMutator(AcidOutputFormat<?, ?> outputFormat, long transactionId, Path partitionPath, int bucketId)
+ throws IOException {
+ return new MutatorImpl(configuration, recordIdColumn, objectInspector, outputFormat, transactionId, partitionPath,
+ bucketId);
+ }
+
+ @Override
+ public RecordInspector newRecordInspector() {
+ return new RecordInspectorImpl(objectInspector, recordIdColumn);
+ }
+
+ @Override
+ public BucketIdResolver newBucketIdResolver(int totalBuckets) {
+ return new BucketIdResolverImpl(objectInspector, recordIdColumn, totalBuckets, bucketColumnIndexes);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java
new file mode 100644
index 0000000..477ed8c
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingAssert.java
@@ -0,0 +1,191 @@
+/**
+ * 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.hive.hcatalog.streaming.mutate;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.ValidTxnList;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.io.AcidInputFormat.AcidRecordReader;
+import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.ql.io.AcidUtils.Directory;
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcStruct;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.thrift.TException;
+
+public class StreamingAssert {
+
+ public static class Factory {
+ private IMetaStoreClient metaStoreClient;
+ private final HiveConf conf;
+
+ public Factory(IMetaStoreClient metaStoreClient, HiveConf conf) {
+ this.metaStoreClient = metaStoreClient;
+ this.conf = conf;
+ }
+
+ public StreamingAssert newStreamingAssert(Table table) throws Exception {
+ return newStreamingAssert(table, Collections.<String> emptyList());
+ }
+
+ public StreamingAssert newStreamingAssert(Table table, List<String> partition) throws Exception {
+ return new StreamingAssert(metaStoreClient, conf, table, partition);
+ }
+ }
+
+ private Table table;
+ private List<String> partition;
+ private IMetaStoreClient metaStoreClient;
+ private Directory dir;
+ private ValidTxnList txns;
+ private List<AcidUtils.ParsedDelta> currentDeltas;
+ private long min;
+ private long max;
+ private Path partitionLocation;
+
+ StreamingAssert(IMetaStoreClient metaStoreClient, HiveConf conf, Table table, List<String> partition)
+ throws Exception {
+ this.metaStoreClient = metaStoreClient;
+ this.table = table;
+ this.partition = partition;
+
+ txns = metaStoreClient.getValidTxns();
+ partitionLocation = getPartitionLocation();
+ dir = AcidUtils.getAcidState(partitionLocation, conf, txns);
+ assertEquals(0, dir.getObsolete().size());
+ assertEquals(0, dir.getOriginalFiles().size());
+
+ currentDeltas = dir.getCurrentDirectories();
+ min = Long.MAX_VALUE;
+ max = Long.MIN_VALUE;
+ System.out.println("Files found: ");
+ for (AcidUtils.ParsedDelta parsedDelta : currentDeltas) {
+ System.out.println(parsedDelta.getPath().toString());
+ max = Math.max(parsedDelta.getMaxTransaction(), max);
+ min = Math.min(parsedDelta.getMinTransaction(), min);
+ }
+ }
+
+ public void assertExpectedFileCount(int expectedFileCount) {
+ assertEquals(expectedFileCount, currentDeltas.size());
+ }
+
+ public void assertNothingWritten() {
+ assertExpectedFileCount(0);
+ }
+
+ public void assertMinTransactionId(long expectedMinTransactionId) {
+ if (currentDeltas.isEmpty()) {
+ throw new AssertionError("No data");
+ }
+ assertEquals(expectedMinTransactionId, min);
+ }
+
+ public void assertMaxTransactionId(long expectedMaxTransactionId) {
+ if (currentDeltas.isEmpty()) {
+ throw new AssertionError("No data");
+ }
+ assertEquals(expectedMaxTransactionId, max);
+ }
+
+ List<Record> readRecords() throws Exception {
+ if (currentDeltas.isEmpty()) {
+ throw new AssertionError("No data");
+ }
+ InputFormat<NullWritable, OrcStruct> inputFormat = new OrcInputFormat();
+ JobConf job = new JobConf();
+ job.set("mapred.input.dir", partitionLocation.toString());
+ job.set("bucket_count", Integer.toString(table.getSd().getNumBuckets()));
+ job.set(ValidTxnList.VALID_TXNS_KEY, txns.toString());
+ InputSplit[] splits = inputFormat.getSplits(job, 1);
+ assertEquals(1, splits.length);
+
+ final AcidRecordReader<NullWritable, OrcStruct> recordReader = (AcidRecordReader<NullWritable, OrcStruct>) inputFormat
+ .getRecordReader(splits[0], job, Reporter.NULL);
+
+ NullWritable key = recordReader.createKey();
+ OrcStruct value = recordReader.createValue();
+
+ List<Record> records = new ArrayList<>();
+ while (recordReader.next(key, value)) {
+ RecordIdentifier recordIdentifier = recordReader.getRecordIdentifier();
+ Record record = new Record(new RecordIdentifier(recordIdentifier.getTransactionId(),
+ recordIdentifier.getBucketId(), recordIdentifier.getRowId()), value.toString());
+ System.out.println(record);
+ records.add(record);
+ }
+ recordReader.close();
+ return records;
+ }
+
+ private Path getPartitionLocation() throws NoSuchObjectException, MetaException, TException {
+ Path partitionLocacation;
+ if (partition.isEmpty()) {
+ partitionLocacation = new Path(table.getSd().getLocation());
+ } else {
+ // TODO: calculate this instead. Just because we're writing to the location doesn't mean that it'll
+ // always be wanted in the meta store right away.
+ List<Partition> partitionEntries = metaStoreClient.listPartitions(table.getDbName(), table.getTableName(),
+ partition, (short) 1);
+ partitionLocacation = new Path(partitionEntries.get(0).getSd().getLocation());
+ }
+ return partitionLocacation;
+ }
+
+ public static class Record {
+ private RecordIdentifier recordIdentifier;
+ private String row;
+
+ Record(RecordIdentifier recordIdentifier, String row) {
+ this.recordIdentifier = recordIdentifier;
+ this.row = row;
+ }
+
+ public RecordIdentifier getRecordIdentifier() {
+ return recordIdentifier;
+ }
+
+ public String getRow() {
+ return row;
+ }
+
+ @Override
+ public String toString() {
+ return "Record [recordIdentifier=" + recordIdentifier + ", row=" + row + "]";
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java
new file mode 100644
index 0000000..f8c8537
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/StreamingTestUtils.java
@@ -0,0 +1,261 @@
+package org.apache.hive.hcatalog.streaming.mutate;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.Warehouse;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
+import org.apache.hadoop.hive.ql.io.HiveInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.thrift.TException;
+
+public class StreamingTestUtils {
+
+ public HiveConf newHiveConf(String metaStoreUri) {
+ HiveConf conf = new HiveConf(this.getClass());
+ conf.set("fs.raw.impl", RawFileSystem.class.getName());
+ if (metaStoreUri != null) {
+ conf.setVar(HiveConf.ConfVars.METASTOREURIS, metaStoreUri);
+ }
+ conf.setBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI, true);
+ conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, true);
+ return conf;
+ }
+
+ public void prepareTransactionDatabase(HiveConf conf) throws Exception {
+ TxnDbUtil.setConfValues(conf);
+ TxnDbUtil.cleanDb();
+ TxnDbUtil.prepDb();
+ }
+
+ public IMetaStoreClient newMetaStoreClient(HiveConf conf) throws Exception {
+ return new HiveMetaStoreClient(conf);
+ }
+
+ public static class RawFileSystem extends RawLocalFileSystem {
+ private static final URI NAME;
+ static {
+ try {
+ NAME = new URI("raw:///");
+ } catch (URISyntaxException se) {
+ throw new IllegalArgumentException("bad uri", se);
+ }
+ }
+
+ @Override
+ public URI getUri() {
+ return NAME;
+ }
+
+ @Override
+ public FileStatus getFileStatus(Path path) throws IOException {
+ File file = pathToFile(path);
+ if (!file.exists()) {
+ throw new FileNotFoundException("Can't find " + path);
+ }
+ // get close enough
+ short mod = 0;
+ if (file.canRead()) {
+ mod |= 0444;
+ }
+ if (file.canWrite()) {
+ mod |= 0200;
+ }
+ if (file.canExecute()) {
+ mod |= 0111;
+ }
+ return new FileStatus(file.length(), file.isDirectory(), 1, 1024, file.lastModified(), file.lastModified(),
+ FsPermission.createImmutable(mod), "owen", "users", path);
+ }
+ }
+
+ public static DatabaseBuilder databaseBuilder(File warehouseFolder) {
+ return new DatabaseBuilder(warehouseFolder);
+ }
+
+ public static class DatabaseBuilder {
+
+ private Database database;
+ private File warehouseFolder;
+
+ public DatabaseBuilder(File warehouseFolder) {
+ this.warehouseFolder = warehouseFolder;
+ database = new Database();
+ }
+
+ public DatabaseBuilder name(String name) {
+ database.setName(name);
+ File databaseFolder = new File(warehouseFolder, name + ".db");
+ String databaseLocation = "raw://" + databaseFolder.toURI().getPath();
+ database.setLocationUri(databaseLocation);
+ return this;
+ }
+
+ public Database dropAndCreate(IMetaStoreClient metaStoreClient) throws Exception {
+ if (metaStoreClient == null) {
+ throw new IllegalArgumentException();
+ }
+ try {
+ for (String table : metaStoreClient.listTableNamesByFilter(database.getName(), "", (short) -1)) {
+ metaStoreClient.dropTable(database.getName(), table, true, true);
+ }
+ metaStoreClient.dropDatabase(database.getName());
+ } catch (TException e) {
+ }
+ metaStoreClient.createDatabase(database);
+ return database;
+ }
+
+ public Database build() {
+ return database;
+ }
+
+ }
+
+ public static TableBuilder tableBuilder(Database database) {
+ return new TableBuilder(database);
+ }
+
+ public static class TableBuilder {
+
+ private Table table;
+ private StorageDescriptor sd;
+ private SerDeInfo serDeInfo;
+ private Database database;
+ private List<List<String>> partitions;
+ private List<String> columnNames;
+ private List<String> columnTypes;
+ private List<String> partitionKeys;
+
+ public TableBuilder(Database database) {
+ this.database = database;
+ partitions = new ArrayList<>();
+ columnNames = new ArrayList<>();
+ columnTypes = new ArrayList<>();
+ partitionKeys = Collections.emptyList();
+ table = new Table();
+ table.setDbName(database.getName());
+ table.setTableType(TableType.MANAGED_TABLE.toString());
+ Map<String, String> tableParams = new HashMap<String, String>();
+ tableParams.put("transactional", Boolean.TRUE.toString());
+ table.setParameters(tableParams);
+
+ sd = new StorageDescriptor();
+ sd.setInputFormat(HiveInputFormat.class.getName());
+ sd.setOutputFormat(OrcOutputFormat.class.getName());
+ sd.setNumBuckets(1);
+ table.setSd(sd);
+
+ serDeInfo = new SerDeInfo();
+ serDeInfo.setParameters(new HashMap<String, String>());
+ serDeInfo.getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1");
+ serDeInfo.setSerializationLib(OrcSerde.class.getName());
+ sd.setSerdeInfo(serDeInfo);
+ }
+
+ public TableBuilder name(String name) {
+ sd.setLocation(database.getLocationUri() + Path.SEPARATOR + name);
+ table.setTableName(name);
+ serDeInfo.setName(name);
+ return this;
+ }
+
+ public TableBuilder buckets(int buckets) {
+ sd.setNumBuckets(buckets);
+ return this;
+ }
+
+ public TableBuilder addColumn(String columnName, String columnType) {
+ columnNames.add(columnName);
+ columnTypes.add(columnType);
+ return this;
+ }
+
+ public TableBuilder partitionKeys(String... partitionKeys) {
+ this.partitionKeys = Arrays.asList(partitionKeys);
+ return this;
+ }
+
+ public TableBuilder addPartition(String... partitionValues) {
+ partitions.add(Arrays.asList(partitionValues));
+ return this;
+ }
+
+ public TableBuilder addPartition(List<String> partitionValues) {
+ partitions.add(partitionValues);
+ return this;
+ }
+
+ public Table create(IMetaStoreClient metaStoreClient) throws Exception {
+ if (metaStoreClient == null) {
+ throw new IllegalArgumentException();
+ }
+ return internalCreate(metaStoreClient);
+ }
+
+ public Table build() throws Exception {
+ return internalCreate(null);
+ }
+
+ private Table internalCreate(IMetaStoreClient metaStoreClient) throws Exception {
+ List<FieldSchema> fields = new ArrayList<FieldSchema>(columnNames.size());
+ for (int i = 0; i < columnNames.size(); i++) {
+ fields.add(new FieldSchema(columnNames.get(i), columnTypes.get(i), ""));
+ }
+ sd.setCols(fields);
+
+ if (!partitionKeys.isEmpty()) {
+ List<FieldSchema> partitionFields = new ArrayList<FieldSchema>();
+ for (String partitionKey : partitionKeys) {
+ partitionFields.add(new FieldSchema(partitionKey, serdeConstants.STRING_TYPE_NAME, ""));
+ }
+ table.setPartitionKeys(partitionFields);
+ }
+ if (metaStoreClient != null) {
+ metaStoreClient.createTable(table);
+ }
+
+ for (List<String> partitionValues : partitions) {
+ Partition partition = new Partition();
+ partition.setDbName(database.getName());
+ partition.setTableName(table.getTableName());
+ StorageDescriptor partitionSd = new StorageDescriptor(table.getSd());
+ partitionSd.setLocation(table.getSd().getLocation() + Path.SEPARATOR
+ + Warehouse.makePartName(table.getPartitionKeys(), partitionValues));
+ partition.setSd(partitionSd);
+ partition.setValues(partitionValues);
+
+ if (metaStoreClient != null) {
+ metaStoreClient.add_partition(partition);
+ }
+ }
+ return table;
+ }
+ }
+
+}
[05/25] hive git commit: HIVE-10165 Improve hive-hcatalog-streaming
extensibility and support updates and deletes (Eliot West via gates)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java
new file mode 100644
index 0000000..703cef6
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/TestMutations.java
@@ -0,0 +1,544 @@
+/**
+ * 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.hive.hcatalog.streaming.mutate;
+
+import static org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState.ABORTED;
+import static org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState.COMMITTED;
+import static org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils.databaseBuilder;
+import static org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils.tableBuilder;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hive.hcatalog.streaming.TestStreaming;
+import org.apache.hive.hcatalog.streaming.mutate.StreamingAssert.Factory;
+import org.apache.hive.hcatalog.streaming.mutate.StreamingAssert.Record;
+import org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils.TableBuilder;
+import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClient;
+import org.apache.hive.hcatalog.streaming.mutate.client.MutatorClientBuilder;
+import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable;
+import org.apache.hive.hcatalog.streaming.mutate.client.Transaction;
+import org.apache.hive.hcatalog.streaming.mutate.worker.BucketIdResolver;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinator;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorCoordinatorBuilder;
+import org.apache.hive.hcatalog.streaming.mutate.worker.MutatorFactory;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * This test is based on {@link TestStreaming} and has a similar core set of tests to ensure that basic transactional
+ * behaviour is as expected in the {@link RecordMutator} line. This is complemented with a set of tests related to the
+ * use of update and delete operations.
+ */
+public class TestMutations {
+
+ private static final List<String> EUROPE_FRANCE = Arrays.asList("Europe", "France");
+ private static final List<String> EUROPE_UK = Arrays.asList("Europe", "UK");
+ private static final List<String> ASIA_INDIA = Arrays.asList("Asia", "India");
+ // id
+ private static final int[] BUCKET_COLUMN_INDEXES = new int[] { 0 };
+ private static final int RECORD_ID_COLUMN = 2;
+
+ @Rule
+ public TemporaryFolder warehouseFolder = new TemporaryFolder();
+
+ private StreamingTestUtils testUtils = new StreamingTestUtils();
+ private HiveConf conf;
+ private IMetaStoreClient metaStoreClient;
+ private String metaStoreUri;
+ private Database database;
+ private TableBuilder partitionedTableBuilder;
+ private TableBuilder unpartitionedTableBuilder;
+ private Factory assertionFactory;
+
+ public TestMutations() throws Exception {
+ conf = testUtils.newHiveConf(metaStoreUri);
+ testUtils.prepareTransactionDatabase(conf);
+ metaStoreClient = testUtils.newMetaStoreClient(conf);
+ assertionFactory = new StreamingAssert.Factory(metaStoreClient, conf);
+ }
+
+ @Before
+ public void setup() throws Exception {
+ database = databaseBuilder(warehouseFolder.getRoot()).name("testing").dropAndCreate(metaStoreClient);
+
+ partitionedTableBuilder = tableBuilder(database)
+ .name("partitioned")
+ .addColumn("id", "int")
+ .addColumn("msg", "string")
+ .partitionKeys("continent", "country");
+
+ unpartitionedTableBuilder = tableBuilder(database)
+ .name("unpartitioned")
+ .addColumn("id", "int")
+ .addColumn("msg", "string");
+ }
+
+ @Test
+ public void testTransactionBatchEmptyCommitPartitioned() throws Exception {
+ Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), true)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ transaction.begin();
+
+ transaction.commit();
+ assertThat(transaction.getState(), is(COMMITTED));
+ client.close();
+ }
+
+ @Test
+ public void testTransactionBatchEmptyCommitUnpartitioned() throws Exception {
+ Table table = unpartitionedTableBuilder.create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), false)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ transaction.begin();
+
+ transaction.commit();
+ assertThat(transaction.getState(), is(COMMITTED));
+ client.close();
+ }
+
+ @Test
+ public void testTransactionBatchEmptyAbortPartitioned() throws Exception {
+ Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), true)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ List<AcidTable> destinations = client.getTables();
+
+ transaction.begin();
+
+ MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN,
+ BUCKET_COLUMN_INDEXES);
+ MutatorCoordinator coordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ coordinator.close();
+
+ transaction.abort();
+ assertThat(transaction.getState(), is(ABORTED));
+ client.close();
+ }
+
+ @Test
+ public void testTransactionBatchEmptyAbortUnartitioned() throws Exception {
+ Table table = unpartitionedTableBuilder.create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), false)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ List<AcidTable> destinations = client.getTables();
+
+ transaction.begin();
+
+ MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN,
+ BUCKET_COLUMN_INDEXES);
+ MutatorCoordinator coordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ coordinator.close();
+
+ transaction.abort();
+ assertThat(transaction.getState(), is(ABORTED));
+ client.close();
+ }
+
+ @Test
+ public void testTransactionBatchCommitPartitioned() throws Exception {
+ Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), true)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ List<AcidTable> destinations = client.getTables();
+
+ transaction.begin();
+
+ MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN,
+ BUCKET_COLUMN_INDEXES);
+ MutatorCoordinator coordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ BucketIdResolver bucketIdAppender = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets());
+ MutableRecord record = (MutableRecord) bucketIdAppender.attachBucketIdToRecord(new MutableRecord(1,
+ "Hello streaming"));
+ coordinator.insert(ASIA_INDIA, record);
+ coordinator.close();
+
+ transaction.commit();
+
+ StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA);
+ streamingAssertions.assertMinTransactionId(1L);
+ streamingAssertions.assertMaxTransactionId(1L);
+ streamingAssertions.assertExpectedFileCount(1);
+
+ List<Record> readRecords = streamingAssertions.readRecords();
+ assertThat(readRecords.size(), is(1));
+ assertThat(readRecords.get(0).getRow(), is("{1, Hello streaming}"));
+ assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 0L)));
+
+ assertThat(transaction.getState(), is(COMMITTED));
+ client.close();
+ }
+
+ @Test
+ public void testMulti() throws Exception {
+ Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), true)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ List<AcidTable> destinations = client.getTables();
+
+ transaction.begin();
+
+ MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN,
+ BUCKET_COLUMN_INDEXES);
+ MutatorCoordinator coordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets());
+ MutableRecord asiaIndiaRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1,
+ "Hello streaming"));
+ MutableRecord europeUkRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(2,
+ "Hello streaming"));
+ MutableRecord europeFranceRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(3,
+ "Hello streaming"));
+ MutableRecord europeFranceRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(4,
+ "Bonjour streaming"));
+
+ coordinator.insert(ASIA_INDIA, asiaIndiaRecord1);
+ coordinator.insert(EUROPE_UK, europeUkRecord1);
+ coordinator.insert(EUROPE_FRANCE, europeFranceRecord1);
+ coordinator.insert(EUROPE_FRANCE, europeFranceRecord2);
+ coordinator.close();
+
+ transaction.commit();
+
+ // ASIA_INDIA
+ StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA);
+ streamingAssertions.assertMinTransactionId(1L);
+ streamingAssertions.assertMaxTransactionId(1L);
+ streamingAssertions.assertExpectedFileCount(1);
+
+ List<Record> readRecords = streamingAssertions.readRecords();
+ assertThat(readRecords.size(), is(1));
+ assertThat(readRecords.get(0).getRow(), is("{1, Hello streaming}"));
+ assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 0L)));
+
+ // EUROPE_UK
+ streamingAssertions = assertionFactory.newStreamingAssert(table, EUROPE_UK);
+ streamingAssertions.assertMinTransactionId(1L);
+ streamingAssertions.assertMaxTransactionId(1L);
+ streamingAssertions.assertExpectedFileCount(1);
+
+ readRecords = streamingAssertions.readRecords();
+ assertThat(readRecords.size(), is(1));
+ assertThat(readRecords.get(0).getRow(), is("{2, Hello streaming}"));
+ assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 0L)));
+
+ // EUROPE_FRANCE
+ streamingAssertions = assertionFactory.newStreamingAssert(table, EUROPE_FRANCE);
+ streamingAssertions.assertMinTransactionId(1L);
+ streamingAssertions.assertMaxTransactionId(1L);
+ streamingAssertions.assertExpectedFileCount(1);
+
+ readRecords = streamingAssertions.readRecords();
+ assertThat(readRecords.size(), is(2));
+ assertThat(readRecords.get(0).getRow(), is("{3, Hello streaming}"));
+ assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 0L)));
+ assertThat(readRecords.get(1).getRow(), is("{4, Bonjour streaming}"));
+ assertThat(readRecords.get(1).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 1L)));
+
+ client.close();
+ }
+
+ @Test
+ public void testTransactionBatchCommitUnpartitioned() throws Exception {
+ Table table = unpartitionedTableBuilder.create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), false)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ List<AcidTable> destinations = client.getTables();
+
+ transaction.begin();
+
+ MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN,
+ BUCKET_COLUMN_INDEXES);
+ MutatorCoordinator coordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets());
+ MutableRecord record = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1,
+ "Hello streaming"));
+
+ coordinator.insert(Collections.<String> emptyList(), record);
+ coordinator.close();
+
+ transaction.commit();
+
+ StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table);
+ streamingAssertions.assertMinTransactionId(1L);
+ streamingAssertions.assertMaxTransactionId(1L);
+ streamingAssertions.assertExpectedFileCount(1);
+
+ List<Record> readRecords = streamingAssertions.readRecords();
+ assertThat(readRecords.size(), is(1));
+ assertThat(readRecords.get(0).getRow(), is("{1, Hello streaming}"));
+ assertThat(readRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 0L)));
+
+ assertThat(transaction.getState(), is(COMMITTED));
+ client.close();
+ }
+
+ @Test
+ public void testTransactionBatchAbort() throws Exception {
+ Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), true)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction transaction = client.newTransaction();
+
+ List<AcidTable> destinations = client.getTables();
+
+ transaction.begin();
+
+ MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN,
+ BUCKET_COLUMN_INDEXES);
+ MutatorCoordinator coordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets());
+ MutableRecord record1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1,
+ "Hello streaming"));
+ MutableRecord record2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(2,
+ "Welcome to streaming"));
+
+ coordinator.insert(ASIA_INDIA, record1);
+ coordinator.insert(ASIA_INDIA, record2);
+ coordinator.close();
+
+ transaction.abort();
+
+ assertThat(transaction.getState(), is(ABORTED));
+
+ client.close();
+
+ StreamingAssert streamingAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA);
+ streamingAssertions.assertNothingWritten();
+ }
+
+ @Test
+ public void testUpdatesAndDeletes() throws Exception {
+ // Set up some base data then stream some inserts/updates/deletes to a number of partitions
+ MutatorFactory mutatorFactory = new ReflectiveMutatorFactory(conf, MutableRecord.class, RECORD_ID_COLUMN,
+ BUCKET_COLUMN_INDEXES);
+
+ // INSERT DATA
+ //
+ Table table = partitionedTableBuilder.addPartition(ASIA_INDIA).addPartition(EUROPE_FRANCE).create(metaStoreClient);
+
+ MutatorClient client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), true)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction insertTransaction = client.newTransaction();
+
+ List<AcidTable> destinations = client.getTables();
+
+ insertTransaction.begin();
+
+ MutatorCoordinator insertCoordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ BucketIdResolver bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets());
+ MutableRecord asiaIndiaRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(1,
+ "Namaste streaming 1"));
+ MutableRecord asiaIndiaRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(2,
+ "Namaste streaming 2"));
+ MutableRecord europeUkRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(3,
+ "Hello streaming 1"));
+ MutableRecord europeUkRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(4,
+ "Hello streaming 2"));
+ MutableRecord europeFranceRecord1 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(5,
+ "Bonjour streaming 1"));
+ MutableRecord europeFranceRecord2 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(6,
+ "Bonjour streaming 2"));
+
+ insertCoordinator.insert(ASIA_INDIA, asiaIndiaRecord1);
+ insertCoordinator.insert(ASIA_INDIA, asiaIndiaRecord2);
+ insertCoordinator.insert(EUROPE_UK, europeUkRecord1);
+ insertCoordinator.insert(EUROPE_UK, europeUkRecord2);
+ insertCoordinator.insert(EUROPE_FRANCE, europeFranceRecord1);
+ insertCoordinator.insert(EUROPE_FRANCE, europeFranceRecord2);
+ insertCoordinator.close();
+
+ insertTransaction.commit();
+
+ assertThat(insertTransaction.getState(), is(COMMITTED));
+ client.close();
+
+ // MUTATE DATA
+ //
+ client = new MutatorClientBuilder()
+ .addSinkTable(table.getDbName(), table.getTableName(), true)
+ .metaStoreUri(metaStoreUri)
+ .build();
+ client.connect();
+
+ Transaction mutateTransaction = client.newTransaction();
+
+ destinations = client.getTables();
+
+ mutateTransaction.begin();
+
+ MutatorCoordinator mutateCoordinator = new MutatorCoordinatorBuilder()
+ .metaStoreUri(metaStoreUri)
+ .table(destinations.get(0))
+ .mutatorFactory(mutatorFactory)
+ .build();
+
+ bucketIdResolver = mutatorFactory.newBucketIdResolver(destinations.get(0).getTotalBuckets());
+ MutableRecord asiaIndiaRecord3 = (MutableRecord) bucketIdResolver.attachBucketIdToRecord(new MutableRecord(20,
+ "Namaste streaming 3"));
+
+ mutateCoordinator.update(ASIA_INDIA, new MutableRecord(2, "UPDATED: Namaste streaming 2", new RecordIdentifier(1L,
+ 0, 1L)));
+ mutateCoordinator.insert(ASIA_INDIA, asiaIndiaRecord3);
+ mutateCoordinator.delete(EUROPE_UK, new MutableRecord(3, "Hello streaming 1", new RecordIdentifier(1L, 0, 0L)));
+ mutateCoordinator.delete(EUROPE_FRANCE,
+ new MutableRecord(5, "Bonjour streaming 1", new RecordIdentifier(1L, 0, 0L)));
+ mutateCoordinator.update(EUROPE_FRANCE, new MutableRecord(6, "UPDATED: Bonjour streaming 2", new RecordIdentifier(
+ 1L, 0, 1L)));
+ mutateCoordinator.close();
+
+ mutateTransaction.commit();
+
+ assertThat(mutateTransaction.getState(), is(COMMITTED));
+
+ StreamingAssert indiaAssertions = assertionFactory.newStreamingAssert(table, ASIA_INDIA);
+ indiaAssertions.assertMinTransactionId(1L);
+ indiaAssertions.assertMaxTransactionId(2L);
+ List<Record> indiaRecords = indiaAssertions.readRecords();
+ assertThat(indiaRecords.size(), is(3));
+ assertThat(indiaRecords.get(0).getRow(), is("{1, Namaste streaming 1}"));
+ assertThat(indiaRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 0L)));
+ assertThat(indiaRecords.get(1).getRow(), is("{2, UPDATED: Namaste streaming 2}"));
+ assertThat(indiaRecords.get(1).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 1L)));
+ assertThat(indiaRecords.get(2).getRow(), is("{20, Namaste streaming 3}"));
+ assertThat(indiaRecords.get(2).getRecordIdentifier(), is(new RecordIdentifier(2L, 0, 0L)));
+
+ StreamingAssert ukAssertions = assertionFactory.newStreamingAssert(table, EUROPE_UK);
+ ukAssertions.assertMinTransactionId(1L);
+ ukAssertions.assertMaxTransactionId(2L);
+ List<Record> ukRecords = ukAssertions.readRecords();
+ assertThat(ukRecords.size(), is(1));
+ assertThat(ukRecords.get(0).getRow(), is("{4, Hello streaming 2}"));
+ assertThat(ukRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 1L)));
+
+ StreamingAssert franceAssertions = assertionFactory.newStreamingAssert(table, EUROPE_FRANCE);
+ franceAssertions.assertMinTransactionId(1L);
+ franceAssertions.assertMaxTransactionId(2L);
+ List<Record> franceRecords = franceAssertions.readRecords();
+ assertThat(franceRecords.size(), is(1));
+ assertThat(franceRecords.get(0).getRow(), is("{6, UPDATED: Bonjour streaming 2}"));
+ assertThat(franceRecords.get(0).getRecordIdentifier(), is(new RecordIdentifier(1L, 0, 1L)));
+
+ client.close();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java
new file mode 100644
index 0000000..706697a
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestAcidTableSerializer.java
@@ -0,0 +1,66 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.junit.Assert.assertThat;
+
+import java.io.File;
+
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hive.hcatalog.streaming.mutate.StreamingTestUtils;
+import org.junit.Test;
+
+public class TestAcidTableSerializer {
+
+ @Test
+ public void testSerializeDeserialize() throws Exception {
+ Database database = StreamingTestUtils.databaseBuilder(new File("/tmp")).name("db_1").build();
+ Table table = StreamingTestUtils
+ .tableBuilder(database)
+ .name("table_1")
+ .addColumn("one", "string")
+ .addColumn("two", "integer")
+ .partitionKeys("partition")
+ .addPartition("p1")
+ .buckets(10)
+ .build();
+
+ AcidTable acidTable = new AcidTable("db_1", "table_1", true, TableType.SINK);
+ acidTable.setTable(table);
+ acidTable.setTransactionId(42L);
+
+ String encoded = AcidTableSerializer.encode(acidTable);
+ System.out.println(encoded);
+ AcidTable decoded = AcidTableSerializer.decode(encoded);
+
+ assertThat(decoded.getDatabaseName(), is("db_1"));
+ assertThat(decoded.getTableName(), is("table_1"));
+ assertThat(decoded.createPartitions(), is(true));
+ assertThat(decoded.getOutputFormatName(), is("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat"));
+ assertThat(decoded.getTotalBuckets(), is(10));
+ assertThat(decoded.getQualifiedName(), is("DB_1.TABLE_1"));
+ assertThat(decoded.getTransactionId(), is(42L));
+ assertThat(decoded.getTableType(), is(TableType.SINK));
+ assertThat(decoded.getTable(), is(table));
+ }
+
+ @Test
+ public void testSerializeDeserializeNoTableNoTransaction() throws Exception {
+ AcidTable acidTable = new AcidTable("db_1", "table_1", true, TableType.SINK);
+
+ String encoded = AcidTableSerializer.encode(acidTable);
+ AcidTable decoded = AcidTableSerializer.decode(encoded);
+
+ assertThat(decoded.getDatabaseName(), is("db_1"));
+ assertThat(decoded.getTableName(), is("table_1"));
+ assertThat(decoded.createPartitions(), is(true));
+ assertThat(decoded.getOutputFormatName(), is(nullValue()));
+ assertThat(decoded.getTotalBuckets(), is(0));
+ assertThat(decoded.getQualifiedName(), is("DB_1.TABLE_1"));
+ assertThat(decoded.getTransactionId(), is(0L));
+ assertThat(decoded.getTableType(), is(TableType.SINK));
+ assertThat(decoded.getTable(), is(nullValue()));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java
new file mode 100644
index 0000000..ca3f7b2
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestMutatorClient.java
@@ -0,0 +1,176 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
+import org.apache.hive.hcatalog.streaming.TransactionBatch.TxnState;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockFailureListener;
+import org.apache.thrift.TException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestMutatorClient {
+
+ private static final long TRANSACTION_ID = 42L;
+ private static final String TABLE_NAME_1 = "TABLE_1";
+ private static final String TABLE_NAME_2 = "TABLE_2";
+ private static final String DB_NAME = "DB_1";
+ private static final String USER = "user";
+ private static final AcidTable TABLE_1 = new AcidTable(DB_NAME, TABLE_NAME_1, true, TableType.SINK);
+ private static final AcidTable TABLE_2 = new AcidTable(DB_NAME, TABLE_NAME_2, true, TableType.SINK);
+
+ @Mock
+ private IMetaStoreClient mockMetaStoreClient;
+ @Mock
+ private Lock mockLock;
+ @Mock
+ private Table mockTable1, mockTable2;
+ @Mock
+ private StorageDescriptor mockSd;
+ @Mock
+ private Map<String, String> mockParameters;
+ @Mock
+ private HiveConf mockConfiguration;
+ @Mock
+ private LockFailureListener mockLockFailureListener;
+
+ private MutatorClient client;
+
+ @Before
+ public void configureMocks() throws Exception {
+ when(mockMetaStoreClient.getTable(DB_NAME, TABLE_NAME_1)).thenReturn(mockTable1);
+ when(mockTable1.getDbName()).thenReturn(DB_NAME);
+ when(mockTable1.getTableName()).thenReturn(TABLE_NAME_1);
+ when(mockTable1.getSd()).thenReturn(mockSd);
+ when(mockTable1.getParameters()).thenReturn(mockParameters);
+ when(mockMetaStoreClient.getTable(DB_NAME, TABLE_NAME_2)).thenReturn(mockTable2);
+ when(mockTable2.getDbName()).thenReturn(DB_NAME);
+ when(mockTable2.getTableName()).thenReturn(TABLE_NAME_2);
+ when(mockTable2.getSd()).thenReturn(mockSd);
+ when(mockTable2.getParameters()).thenReturn(mockParameters);
+ when(mockSd.getNumBuckets()).thenReturn(1, 2);
+ when(mockSd.getOutputFormat()).thenReturn(OrcOutputFormat.class.getName());
+ when(mockParameters.get("transactional")).thenReturn(Boolean.TRUE.toString());
+
+ when(mockMetaStoreClient.openTxn(USER)).thenReturn(TRANSACTION_ID);
+
+ client = new MutatorClient(mockMetaStoreClient, mockConfiguration, mockLockFailureListener, USER,
+ Collections.singletonList(TABLE_1));
+ }
+
+ @Test
+ public void testCheckValidTableConnect() throws Exception {
+ List<AcidTable> inTables = new ArrayList<>();
+ inTables.add(TABLE_1);
+ inTables.add(TABLE_2);
+ client = new MutatorClient(mockMetaStoreClient, mockConfiguration, mockLockFailureListener, USER, inTables);
+
+ client.connect();
+ List<AcidTable> outTables = client.getTables();
+
+ assertThat(client.isConnected(), is(true));
+ assertThat(outTables.size(), is(2));
+ assertThat(outTables.get(0).getDatabaseName(), is(DB_NAME));
+ assertThat(outTables.get(0).getTableName(), is(TABLE_NAME_1));
+ assertThat(outTables.get(0).getTotalBuckets(), is(2));
+ assertThat(outTables.get(0).getOutputFormatName(), is(OrcOutputFormat.class.getName()));
+ assertThat(outTables.get(0).getTransactionId(), is(0L));
+ assertThat(outTables.get(0).getTable(), is(mockTable1));
+ assertThat(outTables.get(1).getDatabaseName(), is(DB_NAME));
+ assertThat(outTables.get(1).getTableName(), is(TABLE_NAME_2));
+ assertThat(outTables.get(1).getTotalBuckets(), is(2));
+ assertThat(outTables.get(1).getOutputFormatName(), is(OrcOutputFormat.class.getName()));
+ assertThat(outTables.get(1).getTransactionId(), is(0L));
+ assertThat(outTables.get(1).getTable(), is(mockTable2));
+ }
+
+ @Test
+ public void testCheckNonTransactionalTableConnect() throws Exception {
+ when(mockParameters.get("transactional")).thenReturn(Boolean.FALSE.toString());
+
+ try {
+ client.connect();
+ fail();
+ } catch (ConnectionException e) {
+ }
+
+ assertThat(client.isConnected(), is(false));
+ }
+
+ @Test
+ public void testCheckUnBucketedTableConnect() throws Exception {
+ when(mockSd.getNumBuckets()).thenReturn(0);
+
+ try {
+ client.connect();
+ fail();
+ } catch (ConnectionException e) {
+ }
+
+ assertThat(client.isConnected(), is(false));
+ }
+
+ @Test
+ public void testMetaStoreFailsOnConnect() throws Exception {
+ when(mockMetaStoreClient.getTable(anyString(), anyString())).thenThrow(new TException());
+
+ try {
+ client.connect();
+ fail();
+ } catch (ConnectionException e) {
+ }
+
+ assertThat(client.isConnected(), is(false));
+ }
+
+ @Test(expected = ConnectionException.class)
+ public void testGetDestinationsFailsIfNotConnected() throws Exception {
+ client.getTables();
+ }
+
+ @Test
+ public void testNewTransaction() throws Exception {
+ List<AcidTable> inTables = new ArrayList<>();
+ inTables.add(TABLE_1);
+ inTables.add(TABLE_2);
+ client = new MutatorClient(mockMetaStoreClient, mockConfiguration, mockLockFailureListener, USER, inTables);
+
+ client.connect();
+ Transaction transaction = client.newTransaction();
+ List<AcidTable> outTables = client.getTables();
+
+ assertThat(client.isConnected(), is(true));
+
+ assertThat(transaction.getTransactionId(), is(TRANSACTION_ID));
+ assertThat(transaction.getState(), is(TxnState.INACTIVE));
+ assertThat(outTables.get(0).getTransactionId(), is(TRANSACTION_ID));
+ assertThat(outTables.get(1).getTransactionId(), is(TRANSACTION_ID));
+ }
+
+ @Test
+ public void testCloseClosesClient() throws Exception {
+ client.close();
+ assertThat(client.isConnected(), is(false));
+ verify(mockMetaStoreClient).close();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java
new file mode 100644
index 0000000..179207a
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/TestTransaction.java
@@ -0,0 +1,95 @@
+package org.apache.hive.hcatalog.streaming.mutate.client;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hive.hcatalog.streaming.TransactionBatch;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.Lock;
+import org.apache.hive.hcatalog.streaming.mutate.client.lock.LockException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestTransaction {
+
+ private static final String USER = "user";
+ private static final long TRANSACTION_ID = 10L;
+
+ @Mock
+ private Lock mockLock;
+ @Mock
+ private IMetaStoreClient mockMetaStoreClient;
+
+ private Transaction transaction;
+
+ @Before
+ public void createTransaction() throws Exception {
+ when(mockLock.getUser()).thenReturn(USER);
+ when(mockMetaStoreClient.openTxn(USER)).thenReturn(TRANSACTION_ID);
+ transaction = new Transaction(mockMetaStoreClient, mockLock);
+ }
+
+ @Test
+ public void testInitialState() {
+ assertThat(transaction.getState(), is(TransactionBatch.TxnState.INACTIVE));
+ assertThat(transaction.getTransactionId(), is(TRANSACTION_ID));
+ }
+
+ @Test
+ public void testBegin() throws Exception {
+ transaction.begin();
+
+ verify(mockLock).acquire(TRANSACTION_ID);
+ assertThat(transaction.getState(), is(TransactionBatch.TxnState.OPEN));
+ }
+
+ @Test
+ public void testBeginLockFails() throws Exception {
+ doThrow(new LockException("")).when(mockLock).acquire(TRANSACTION_ID);
+
+ try {
+ transaction.begin();
+ } catch (TransactionException ignore) {
+ }
+
+ assertThat(transaction.getState(), is(TransactionBatch.TxnState.INACTIVE));
+ }
+
+ @Test
+ public void testCommit() throws Exception {
+ transaction.commit();
+
+ verify(mockLock).release();
+ verify(mockMetaStoreClient).commitTxn(TRANSACTION_ID);
+ assertThat(transaction.getState(), is(TransactionBatch.TxnState.COMMITTED));
+ }
+
+ @Test(expected = TransactionException.class)
+ public void testCommitLockFails() throws Exception {
+ doThrow(new LockException("")).when(mockLock).release();
+ transaction.commit();
+ }
+
+ @Test
+ public void testAbort() throws Exception {
+ transaction.abort();
+
+ verify(mockLock).release();
+ verify(mockMetaStoreClient).rollbackTxn(TRANSACTION_ID);
+ assertThat(transaction.getState(), is(TransactionBatch.TxnState.ABORTED));
+ }
+
+ @Test(expected = TransactionException.class)
+ public void testAbortLockFails() throws Exception {
+ doThrow(new LockException("")).when(mockLock).release();
+ transaction.abort();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java
new file mode 100644
index 0000000..8e6d06e
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestHeartbeatTimerTask.java
@@ -0,0 +1,100 @@
+package org.apache.hive.hcatalog.streaming.mutate.client.lock;
+
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.verify;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.thrift.TException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestHeartbeatTimerTask {
+
+ private static final long TRANSACTION_ID = 10L;
+ private static final long LOCK_ID = 1L;
+ private static final List<Table> TABLES = createTable();
+
+ @Mock
+ private IMetaStoreClient mockMetaStoreClient;
+ @Mock
+ private LockFailureListener mockListener;
+
+ private HeartbeatTimerTask task;
+
+ @Before
+ public void create() throws Exception {
+ task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, TABLES, LOCK_ID);
+ }
+
+ @Test
+ public void testRun() throws Exception {
+ task.run();
+
+ verify(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+ }
+
+ @Test
+ public void testRunNullTransactionId() throws Exception {
+ task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, null, TABLES, LOCK_ID);
+
+ task.run();
+
+ verify(mockMetaStoreClient).heartbeat(0, LOCK_ID);
+ }
+
+ @Test
+ public void testRunHeartbeatFailsNoSuchLockException() throws Exception {
+ NoSuchLockException exception = new NoSuchLockException();
+ doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+
+ task.run();
+
+ verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Arrays.asList("DB.TABLE"), exception);
+ }
+
+ @Test
+ public void testRunHeartbeatFailsNoSuchTxnException() throws Exception {
+ NoSuchTxnException exception = new NoSuchTxnException();
+ doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+
+ task.run();
+
+ verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Arrays.asList("DB.TABLE"), exception);
+ }
+
+ @Test
+ public void testRunHeartbeatFailsTxnAbortedException() throws Exception {
+ TxnAbortedException exception = new TxnAbortedException();
+ doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+
+ task.run();
+
+ verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Arrays.asList("DB.TABLE"), exception);
+ }
+
+ @Test
+ public void testRunHeartbeatFailsTException() throws Exception {
+ TException exception = new TException();
+ doThrow(exception).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+
+ task.run();
+ }
+
+ private static List<Table> createTable() {
+ Table table = new Table();
+ table.setDbName("DB");
+ table.setTableName("TABLE");
+ return Arrays.asList(table);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java
new file mode 100644
index 0000000..ef1e80c
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/client/lock/TestLock.java
@@ -0,0 +1,283 @@
+package org.apache.hive.hcatalog.streaming.mutate.client.lock;
+
+import static org.apache.hadoop.hive.metastore.api.LockState.ABORT;
+import static org.apache.hadoop.hive.metastore.api.LockState.ACQUIRED;
+import static org.apache.hadoop.hive.metastore.api.LockState.NOT_ACQUIRED;
+import static org.apache.hadoop.hive.metastore.api.LockState.WAITING;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.List;
+import java.util.Timer;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.LockComponent;
+import org.apache.hadoop.hive.metastore.api.LockLevel;
+import org.apache.hadoop.hive.metastore.api.LockRequest;
+import org.apache.hadoop.hive.metastore.api.LockResponse;
+import org.apache.hadoop.hive.metastore.api.LockType;
+import org.apache.hadoop.hive.metastore.api.NoSuchLockException;
+import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
+import org.apache.thrift.TException;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+import com.google.common.collect.ImmutableList;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestLock {
+
+ private static final Table TABLE_1 = createTable("DB", "ONE");
+ private static final Table TABLE_2 = createTable("DB", "TWO");
+ private static final List<Table> TABLES = ImmutableList.of(TABLE_1, TABLE_2);
+ private static final long LOCK_ID = 42;
+ private static final long TRANSACTION_ID = 109;
+ private static final String USER = "ewest";
+
+ @Mock
+ private IMetaStoreClient mockMetaStoreClient;
+ @Mock
+ private LockFailureListener mockListener;
+ @Mock
+ private LockResponse mockLockResponse;
+ @Mock
+ private HeartbeatFactory mockHeartbeatFactory;
+ @Mock
+ private Timer mockHeartbeat;
+ @Captor
+ private ArgumentCaptor<LockRequest> requestCaptor;
+
+ private Lock lock;
+ private HiveConf configuration = new HiveConf();
+
+ @Before
+ public void injectMocks() throws Exception {
+ when(mockMetaStoreClient.lock(any(LockRequest.class))).thenReturn(mockLockResponse);
+ when(mockLockResponse.getLockid()).thenReturn(LOCK_ID);
+ when(mockLockResponse.getState()).thenReturn(ACQUIRED);
+ when(
+ mockHeartbeatFactory.newInstance(any(IMetaStoreClient.class), any(LockFailureListener.class), any(Long.class),
+ any(Collection.class), anyLong(), anyInt())).thenReturn(mockHeartbeat);
+
+ lock = new Lock(mockMetaStoreClient, mockHeartbeatFactory, configuration, mockListener, USER, TABLES, 3, 0);
+ }
+
+ @Test
+ public void testAcquireReadLockWithNoIssues() throws Exception {
+ lock.acquire();
+ assertEquals(Long.valueOf(LOCK_ID), lock.getLockId());
+ assertNull(lock.getTransactionId());
+ }
+
+ @Test
+ public void testAcquireTxnLockWithNoIssues() throws Exception {
+ lock.acquire(TRANSACTION_ID);
+ assertEquals(Long.valueOf(LOCK_ID), lock.getLockId());
+ assertEquals(Long.valueOf(TRANSACTION_ID), lock.getTransactionId());
+ }
+
+ @Test
+ public void testAcquireReadLockCheckHeartbeatCreated() throws Exception {
+ configuration.set("hive.txn.timeout", "100s");
+ lock.acquire();
+
+ verify(mockHeartbeatFactory).newInstance(eq(mockMetaStoreClient), eq(mockListener), any(Long.class), eq(TABLES),
+ eq(LOCK_ID), eq(75));
+ }
+
+ @Test
+ public void testAcquireTxnLockCheckHeartbeatCreated() throws Exception {
+ configuration.set("hive.txn.timeout", "100s");
+ lock.acquire(TRANSACTION_ID);
+
+ verify(mockHeartbeatFactory).newInstance(eq(mockMetaStoreClient), eq(mockListener), eq(TRANSACTION_ID), eq(TABLES),
+ eq(LOCK_ID), eq(75));
+ }
+
+ @Test
+ public void testAcquireLockCheckUser() throws Exception {
+ lock.acquire();
+ verify(mockMetaStoreClient).lock(requestCaptor.capture());
+ LockRequest actualRequest = requestCaptor.getValue();
+ assertEquals(USER, actualRequest.getUser());
+ }
+
+ @Test
+ public void testAcquireReadLockCheckLocks() throws Exception {
+ lock.acquire();
+ verify(mockMetaStoreClient).lock(requestCaptor.capture());
+
+ LockRequest request = requestCaptor.getValue();
+ assertEquals(0, request.getTxnid());
+ assertEquals(USER, request.getUser());
+ assertEquals(InetAddress.getLocalHost().getHostName(), request.getHostname());
+
+ List<LockComponent> components = request.getComponent();
+
+ assertEquals(2, components.size());
+
+ LockComponent expected1 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
+ expected1.setTablename("ONE");
+ assertTrue(components.contains(expected1));
+
+ LockComponent expected2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
+ expected2.setTablename("TWO");
+ assertTrue(components.contains(expected2));
+ }
+
+ @Test
+ public void testAcquireTxnLockCheckLocks() throws Exception {
+ lock.acquire(TRANSACTION_ID);
+ verify(mockMetaStoreClient).lock(requestCaptor.capture());
+
+ LockRequest request = requestCaptor.getValue();
+ assertEquals(TRANSACTION_ID, request.getTxnid());
+ assertEquals(USER, request.getUser());
+ assertEquals(InetAddress.getLocalHost().getHostName(), request.getHostname());
+
+ List<LockComponent> components = request.getComponent();
+
+ System.out.println(components);
+ assertEquals(2, components.size());
+
+ LockComponent expected1 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
+ expected1.setTablename("ONE");
+ assertTrue(components.contains(expected1));
+
+ LockComponent expected2 = new LockComponent(LockType.SHARED_READ, LockLevel.TABLE, "DB");
+ expected2.setTablename("TWO");
+ assertTrue(components.contains(expected2));
+ }
+
+ @Test(expected = LockException.class)
+ public void testAcquireLockNotAcquired() throws Exception {
+ when(mockLockResponse.getState()).thenReturn(NOT_ACQUIRED);
+ lock.acquire();
+ }
+
+ @Test(expected = LockException.class)
+ public void testAcquireLockAborted() throws Exception {
+ when(mockLockResponse.getState()).thenReturn(ABORT);
+ lock.acquire();
+ }
+
+ @Test(expected = LockException.class)
+ public void testAcquireLockWithWaitRetriesExceeded() throws Exception {
+ when(mockLockResponse.getState()).thenReturn(WAITING, WAITING, WAITING);
+ lock.acquire();
+ }
+
+ @Test
+ public void testAcquireLockWithWaitRetries() throws Exception {
+ when(mockLockResponse.getState()).thenReturn(WAITING, WAITING, ACQUIRED);
+ lock.acquire();
+ assertEquals(Long.valueOf(LOCK_ID), lock.getLockId());
+ }
+
+ @Test
+ public void testReleaseLock() throws Exception {
+ lock.acquire();
+ lock.release();
+ verify(mockMetaStoreClient).unlock(LOCK_ID);
+ }
+
+ @Test
+ public void testReleaseLockNoLock() throws Exception {
+ lock.release();
+ verifyNoMoreInteractions(mockMetaStoreClient);
+ }
+
+ @Test
+ public void testReleaseLockCancelsHeartbeat() throws Exception {
+ lock.acquire();
+ lock.release();
+ verify(mockHeartbeat).cancel();
+ }
+
+ @Test
+ public void testReadHeartbeat() throws Exception {
+ HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, null, TABLES, LOCK_ID);
+ task.run();
+ verify(mockMetaStoreClient).heartbeat(0, LOCK_ID);
+ }
+
+ @Test
+ public void testTxnHeartbeat() throws Exception {
+ HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, TABLES, LOCK_ID);
+ task.run();
+ verify(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+ }
+
+ @Test
+ public void testReadHeartbeatFailsNoSuchLockException() throws Exception {
+ Throwable t = new NoSuchLockException();
+ doThrow(t).when(mockMetaStoreClient).heartbeat(0, LOCK_ID);
+ HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, null, TABLES, LOCK_ID);
+ task.run();
+ verify(mockListener).lockFailed(LOCK_ID, null, Lock.asStrings(TABLES), t);
+ }
+
+ @Test
+ public void testTxnHeartbeatFailsNoSuchLockException() throws Exception {
+ Throwable t = new NoSuchLockException();
+ doThrow(t).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+ HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, TABLES, LOCK_ID);
+ task.run();
+ verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Lock.asStrings(TABLES), t);
+ }
+
+ @Test
+ public void testHeartbeatFailsNoSuchTxnException() throws Exception {
+ Throwable t = new NoSuchTxnException();
+ doThrow(t).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+ HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, TABLES, LOCK_ID);
+ task.run();
+ verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Lock.asStrings(TABLES), t);
+ }
+
+ @Test
+ public void testHeartbeatFailsTxnAbortedException() throws Exception {
+ Throwable t = new TxnAbortedException();
+ doThrow(t).when(mockMetaStoreClient).heartbeat(TRANSACTION_ID, LOCK_ID);
+ HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, TABLES, LOCK_ID);
+ task.run();
+ verify(mockListener).lockFailed(LOCK_ID, TRANSACTION_ID, Lock.asStrings(TABLES), t);
+ }
+
+ @Test
+ public void testHeartbeatContinuesTException() throws Exception {
+ Throwable t = new TException();
+ doThrow(t).when(mockMetaStoreClient).heartbeat(0, LOCK_ID);
+ HeartbeatTimerTask task = new HeartbeatTimerTask(mockMetaStoreClient, mockListener, TRANSACTION_ID, TABLES, LOCK_ID);
+ task.run();
+ verifyZeroInteractions(mockListener);
+ }
+
+ private static Table createTable(String databaseName, String tableName) {
+ Table table = new Table();
+ table.setDbName(databaseName);
+ table.setTableName(tableName);
+ return table;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java
new file mode 100644
index 0000000..f81373e
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestBucketIdResolverImpl.java
@@ -0,0 +1,38 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hive.hcatalog.streaming.mutate.MutableRecord;
+import org.junit.Test;
+
+public class TestBucketIdResolverImpl {
+
+ private static final int TOTAL_BUCKETS = 12;
+ private static final int RECORD_ID_COLUMN = 2;
+ // id - TODO: use a non-zero index to check for offset errors.
+ private static final int[] BUCKET_COLUMN_INDEXES = new int[] { 0 };
+
+ private BucketIdResolver capturingBucketIdResolver = new BucketIdResolverImpl(
+ ObjectInspectorFactory.getReflectionObjectInspector(MutableRecord.class,
+ ObjectInspectorFactory.ObjectInspectorOptions.JAVA), RECORD_ID_COLUMN, TOTAL_BUCKETS, BUCKET_COLUMN_INDEXES);
+
+ @Test
+ public void testAttachBucketIdToRecord() {
+ MutableRecord record = new MutableRecord(1, "hello");
+ capturingBucketIdResolver.attachBucketIdToRecord(record);
+ assertThat(record.rowId, is(new RecordIdentifier(-1L, 8, -1L)));
+ assertThat(record.id, is(1));
+ assertThat(record.msg.toString(), is("hello"));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNoBucketColumns() {
+ new BucketIdResolverImpl(ObjectInspectorFactory.getReflectionObjectInspector(MutableRecord.class,
+ ObjectInspectorFactory.ObjectInspectorOptions.JAVA), RECORD_ID_COLUMN, TOTAL_BUCKETS, new int[0]);
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java
new file mode 100644
index 0000000..74fa59b
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestGroupingValidator.java
@@ -0,0 +1,70 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.junit.Test;
+
+public class TestGroupingValidator {
+
+ private GroupingValidator validator = new GroupingValidator();
+
+ @Test
+ public void uniqueGroups() {
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3));
+ assertTrue(validator.isInSequence(Arrays.asList("b", "B"), 2));
+ }
+
+ @Test
+ public void sameGroup() {
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ }
+
+ @Test
+ public void revisitedGroup() {
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3));
+ assertFalse(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ }
+
+ @Test
+ public void samePartitionDifferentBucket() {
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3));
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 2));
+ }
+
+ @Test
+ public void sameBucketDifferentPartition() {
+ assertTrue(validator.isInSequence(Arrays.asList("a", "A"), 1));
+ assertTrue(validator.isInSequence(Arrays.asList("c", "C"), 3));
+ assertTrue(validator.isInSequence(Arrays.asList("b", "B"), 1));
+ }
+
+ @Test
+ public void uniqueGroupsNoPartition() {
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 1));
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 3));
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 2));
+ }
+
+ @Test
+ public void sameGroupNoPartition() {
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 1));
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 1));
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 1));
+ }
+
+ @Test
+ public void revisitedGroupNoPartition() {
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 1));
+ assertTrue(validator.isInSequence(Collections.<String> emptyList(), 3));
+ assertFalse(validator.isInSequence(Collections.<String> emptyList(), 1));
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java
new file mode 100644
index 0000000..6e9ffa2
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorCoordinator.java
@@ -0,0 +1,234 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
+import org.apache.hive.hcatalog.streaming.mutate.client.AcidTable;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestMutatorCoordinator {
+
+ private static final List<String> UNPARTITIONED = Collections.<String> emptyList();
+ private static final List<String> PARTITION_B = Arrays.asList("B");
+ private static final List<String> PARTITION_A = Arrays.asList("A");
+ private static final long TRANSACTION_ID = 2L;
+ private static final int BUCKET_ID = 0;
+ private static final Path PATH_A = new Path("X");
+ private static final Path PATH_B = new Path("B");
+ private static final Object RECORD = "RECORD";
+ private static final RecordIdentifier ROW__ID_B0_R0 = new RecordIdentifier(10L, BUCKET_ID, 0L);
+ private static final RecordIdentifier ROW__ID_B0_R1 = new RecordIdentifier(10L, BUCKET_ID, 1L);
+ private static final RecordIdentifier ROW__ID_B1_R0 = new RecordIdentifier(10L, BUCKET_ID + 1, 0L);
+ private static final RecordIdentifier ROW__ID_INSERT = new RecordIdentifier(-1L, BUCKET_ID, -1L);
+
+ @Mock
+ private IMetaStoreClient mockMetaStoreClient;
+ @Mock
+ private MutatorFactory mockMutatorFactory;
+ @Mock
+ private CreatePartitionHelper mockPartitionHelper;
+ @Mock
+ private GroupingValidator mockGroupingValidator;
+ @Mock
+ private SequenceValidator mockSequenceValidator;
+ @Mock
+ private AcidTable mockAcidTable;
+ @Mock
+ private RecordInspector mockRecordInspector;
+ @Mock
+ private BucketIdResolver mockBucketIdResolver;
+ @Mock
+ private Mutator mockMutator;
+
+ private MutatorCoordinator coordinator;
+
+ private HiveConf configuration = new HiveConf();
+
+ @Before
+ public void createCoordinator() throws Exception {
+ when(mockAcidTable.getOutputFormatName()).thenReturn(OrcOutputFormat.class.getName());
+ when(mockAcidTable.getTotalBuckets()).thenReturn(1);
+ when(mockAcidTable.getTransactionId()).thenReturn(TRANSACTION_ID);
+ when(mockAcidTable.createPartitions()).thenReturn(true);
+ when(mockMutatorFactory.newRecordInspector()).thenReturn(mockRecordInspector);
+ when(mockMutatorFactory.newBucketIdResolver(anyInt())).thenReturn(mockBucketIdResolver);
+ when(mockMutatorFactory.newMutator(any(OrcOutputFormat.class), anyLong(), any(Path.class), anyInt())).thenReturn(
+ mockMutator);
+ when(mockPartitionHelper.getPathForPartition(any(List.class))).thenReturn(PATH_A);
+ when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_INSERT);
+ when(mockSequenceValidator.isInSequence(any(RecordIdentifier.class))).thenReturn(true);
+ when(mockGroupingValidator.isInSequence(any(List.class), anyInt())).thenReturn(true);
+
+ coordinator = new MutatorCoordinator(mockMetaStoreClient, configuration, mockMutatorFactory, mockPartitionHelper,
+ mockGroupingValidator, mockSequenceValidator, mockAcidTable, false);
+ }
+
+ @Test
+ public void insert() throws Exception {
+ coordinator.insert(UNPARTITIONED, RECORD);
+
+ verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED);
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
+ verify(mockMutator).insert(RECORD);
+ }
+
+ @Test
+ public void multipleInserts() throws Exception {
+ coordinator.insert(UNPARTITIONED, RECORD);
+ coordinator.insert(UNPARTITIONED, RECORD);
+ coordinator.insert(UNPARTITIONED, RECORD);
+
+ verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED);
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
+ verify(mockMutator, times(3)).insert(RECORD);
+ }
+
+ @Test
+ public void insertPartitionChanges() throws Exception {
+ when(mockPartitionHelper.getPathForPartition(PARTITION_A)).thenReturn(PATH_A);
+ when(mockPartitionHelper.getPathForPartition(PARTITION_B)).thenReturn(PATH_B);
+
+ coordinator.insert(PARTITION_A, RECORD);
+ coordinator.insert(PARTITION_B, RECORD);
+
+ verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_A);
+ verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_B);
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_B), eq(BUCKET_ID));
+ verify(mockMutator, times(2)).insert(RECORD);
+ }
+
+ @Test
+ public void bucketChanges() throws Exception {
+ when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0, ROW__ID_B1_R0);
+
+ when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(0, 1);
+
+ coordinator.update(UNPARTITIONED, RECORD);
+ coordinator.delete(UNPARTITIONED, RECORD);
+
+ verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED);
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
+ verify(mockMutatorFactory)
+ .newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID + 1));
+ verify(mockMutator).update(RECORD);
+ verify(mockMutator).delete(RECORD);
+ }
+
+ @Test
+ public void partitionThenBucketChanges() throws Exception {
+ when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0, ROW__ID_B0_R1, ROW__ID_B1_R0,
+ ROW__ID_INSERT);
+
+ when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(0, 0, 1, 0);
+
+ when(mockPartitionHelper.getPathForPartition(PARTITION_A)).thenReturn(PATH_A);
+ when(mockPartitionHelper.getPathForPartition(PARTITION_B)).thenReturn(PATH_B);
+
+ coordinator.update(PARTITION_A, RECORD);
+ coordinator.delete(PARTITION_B, RECORD);
+ coordinator.update(PARTITION_B, RECORD);
+ coordinator.insert(PARTITION_B, RECORD);
+
+ verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_A);
+ verify(mockPartitionHelper).createPartitionIfNotExists(PARTITION_B);
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
+ verify(mockMutatorFactory, times(2)).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_B),
+ eq(BUCKET_ID));
+ verify(mockMutatorFactory)
+ .newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_B), eq(BUCKET_ID + 1));
+ verify(mockMutator, times(2)).update(RECORD);
+ verify(mockMutator).delete(RECORD);
+ verify(mockMutator).insert(RECORD);
+ verify(mockSequenceValidator, times(4)).reset();
+ }
+
+ @Test(expected = RecordSequenceException.class)
+ public void outOfSequence() throws Exception {
+ when(mockSequenceValidator.isInSequence(any(RecordIdentifier.class))).thenReturn(false);
+
+ coordinator.update(UNPARTITIONED, RECORD);
+ coordinator.delete(UNPARTITIONED, RECORD);
+
+ verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED);
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
+ verify(mockMutator).update(RECORD);
+ verify(mockMutator).delete(RECORD);
+ }
+
+ @Test(expected = GroupRevisitedException.class)
+ public void revisitGroup() throws Exception {
+ when(mockGroupingValidator.isInSequence(any(List.class), anyInt())).thenReturn(false);
+
+ coordinator.update(UNPARTITIONED, RECORD);
+ coordinator.delete(UNPARTITIONED, RECORD);
+
+ verify(mockPartitionHelper).createPartitionIfNotExists(UNPARTITIONED);
+ verify(mockMutatorFactory).newMutator(any(OrcOutputFormat.class), eq(TRANSACTION_ID), eq(PATH_A), eq(BUCKET_ID));
+ verify(mockMutator).update(RECORD);
+ verify(mockMutator).delete(RECORD);
+ }
+
+ @Test(expected = BucketIdException.class)
+ public void insertWithBadBucket() throws Exception {
+ when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0);
+
+ when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(1);
+
+ coordinator.insert(UNPARTITIONED, RECORD);
+ }
+
+ @Test(expected = BucketIdException.class)
+ public void updateWithBadBucket() throws Exception {
+ when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0);
+
+ when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(1);
+
+ coordinator.update(UNPARTITIONED, RECORD);
+ }
+
+ @Test
+ public void deleteWithBadBucket() throws Exception {
+ when(mockRecordInspector.extractRecordIdentifier(RECORD)).thenReturn(ROW__ID_B0_R0);
+
+ when(mockBucketIdResolver.computeBucketId(RECORD)).thenReturn(1);
+
+ coordinator.delete(UNPARTITIONED, RECORD);
+ }
+
+ @Test
+ public void closeNoRecords() throws Exception {
+ coordinator.close();
+
+ // No mutator created
+ verifyZeroInteractions(mockMutator);
+ }
+
+ @Test
+ public void closeUsedCoordinator() throws Exception {
+ coordinator.insert(UNPARTITIONED, RECORD);
+ coordinator.close();
+
+ verify(mockMutator).close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java
new file mode 100644
index 0000000..b29c763
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestMutatorImpl.java
@@ -0,0 +1,99 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat;
+import org.apache.hadoop.hive.ql.io.AcidOutputFormat.Options;
+import org.apache.hadoop.hive.ql.io.RecordUpdater;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.ArgumentCaptor;
+import org.mockito.Captor;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestMutatorImpl {
+
+ private static final Object RECORD = new Object();
+ private static final int RECORD_ID_COLUMN = 2;
+ private static final int BUCKET_ID = 0;
+ private static final Path PATH = new Path("X");
+ private static final long TRANSACTION_ID = 1L;
+
+ @Mock
+ private AcidOutputFormat<?, ?> mockOutputFormat;
+ @Mock
+ private ObjectInspector mockObjectInspector;
+ @Mock
+ private RecordUpdater mockRecordUpdater;
+ @Captor
+ private ArgumentCaptor<AcidOutputFormat.Options> captureOptions;
+
+ private final HiveConf configuration = new HiveConf();
+
+ private Mutator mutator;
+
+ @Before
+ public void injectMocks() throws IOException {
+ when(mockOutputFormat.getRecordUpdater(eq(PATH), any(Options.class))).thenReturn(mockRecordUpdater);
+ mutator = new MutatorImpl(configuration, RECORD_ID_COLUMN, mockObjectInspector, mockOutputFormat, TRANSACTION_ID,
+ PATH, BUCKET_ID);
+ }
+
+ @Test
+ public void testCreatesRecordReader() throws IOException {
+ verify(mockOutputFormat).getRecordUpdater(eq(PATH), captureOptions.capture());
+ Options options = captureOptions.getValue();
+ assertThat(options.getBucket(), is(BUCKET_ID));
+ assertThat(options.getConfiguration(), is((Configuration) configuration));
+ assertThat(options.getInspector(), is(mockObjectInspector));
+ assertThat(options.getRecordIdColumn(), is(RECORD_ID_COLUMN));
+ assertThat(options.getMinimumTransactionId(), is(TRANSACTION_ID));
+ assertThat(options.getMaximumTransactionId(), is(TRANSACTION_ID));
+ }
+
+ @Test
+ public void testInsertDelegates() throws IOException {
+ mutator.insert(RECORD);
+ verify(mockRecordUpdater).insert(TRANSACTION_ID, RECORD);
+ }
+
+ @Test
+ public void testUpdateDelegates() throws IOException {
+ mutator.update(RECORD);
+ verify(mockRecordUpdater).update(TRANSACTION_ID, RECORD);
+ }
+
+ @Test
+ public void testDeleteDelegates() throws IOException {
+ mutator.delete(RECORD);
+ verify(mockRecordUpdater).delete(TRANSACTION_ID, RECORD);
+ }
+
+ @Test
+ public void testCloseDelegates() throws IOException {
+ mutator.close();
+ verify(mockRecordUpdater).close(false);
+ }
+
+ @Test
+ public void testFlushDoesNothing() throws IOException {
+ mutator.flush();
+ verify(mockRecordUpdater, never()).flush();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java
new file mode 100644
index 0000000..389ad33
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestRecordInspectorImpl.java
@@ -0,0 +1,31 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hive.hcatalog.streaming.mutate.MutableRecord;
+import org.junit.Test;
+
+public class TestRecordInspectorImpl {
+
+ private static final int ROW_ID_COLUMN = 2;
+
+ private RecordInspectorImpl inspector = new RecordInspectorImpl(ObjectInspectorFactory.getReflectionObjectInspector(
+ MutableRecord.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA), ROW_ID_COLUMN);
+
+ @Test
+ public void testExtractRecordIdentifier() {
+ RecordIdentifier recordIdentifier = new RecordIdentifier(10L, 4, 20L);
+ MutableRecord record = new MutableRecord(1, "hello", recordIdentifier);
+ assertThat(inspector.extractRecordIdentifier(record), is(recordIdentifier));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNotAStructObjectInspector() {
+ new RecordInspectorImpl(PrimitiveObjectInspectorFactory.javaBooleanObjectInspector, 2);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/994d98c0/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java
----------------------------------------------------------------------
diff --git a/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java
new file mode 100644
index 0000000..33f9606
--- /dev/null
+++ b/hcatalog/streaming/src/test/org/apache/hive/hcatalog/streaming/mutate/worker/TestSequenceValidator.java
@@ -0,0 +1,91 @@
+package org.apache.hive.hcatalog.streaming.mutate.worker;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+import org.apache.hadoop.hive.ql.io.RecordIdentifier;
+import org.junit.Test;
+
+public class TestSequenceValidator {
+
+ private static final int BUCKET_ID = 1;
+
+ private SequenceValidator validator = new SequenceValidator();
+
+ @Test
+ public void testSingleInSequence() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ }
+
+ @Test
+ public void testRowIdInSequence() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 4)), is(true));
+ }
+
+ @Test
+ public void testTxIdInSequence() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(4L, BUCKET_ID, 0)), is(true));
+ }
+
+ @Test
+ public void testMixedInSequence() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 1)), is(true));
+ }
+
+ @Test
+ public void testNegativeTxId() {
+ assertThat(validator.isInSequence(new RecordIdentifier(-1L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ }
+
+ @Test
+ public void testNegativeRowId() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, -1)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ }
+
+ @Test
+ public void testRowIdOutOfSequence() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 4)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(false));
+ }
+
+ @Test
+ public void testReset() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 4)), is(true));
+ // New partition for example
+ validator.reset();
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 1)), is(true));
+ }
+
+ @Test
+ public void testTxIdOutOfSequence() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(4L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(false));
+ }
+
+ @Test
+ public void testMixedOutOfSequence() {
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 0)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 4)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 0)), is(false));
+ assertThat(validator.isInSequence(new RecordIdentifier(1L, BUCKET_ID, 5)), is(true));
+ assertThat(validator.isInSequence(new RecordIdentifier(0L, BUCKET_ID, 6)), is(false));
+ }
+
+ @Test(expected = NullPointerException.class)
+ public void testNullRecordIdentifier() {
+ validator.isInSequence(null);
+ }
+
+}
[10/25] hive git commit: HIVE-11055 HPL/SQL - Implementing Procedural
SQL in Hive (PL/HQL Contribution) (Dmitry Tolpeko via gates)
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
new file mode 100644
index 0000000..9895b5e
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/Function.java
@@ -0,0 +1,709 @@
+/**
+ * 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.hive.hplsql.functions;
+
+import java.sql.ResultSet;
+import java.sql.Date;
+import java.sql.SQLException;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang.StringUtils;
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.apache.hive.hplsql.*;
+
+interface FuncCommand {
+ void run(HplsqlParser.Expr_func_paramsContext ctx);
+}
+
+interface FuncSpecCommand {
+ void run(HplsqlParser.Expr_spec_funcContext ctx);
+}
+
+/**
+ * HPL/SQL functions
+ */
+public class Function {
+ Exec exec;
+ HashMap<String, FuncCommand> map = new HashMap<String, FuncCommand>();
+ HashMap<String, FuncSpecCommand> specMap = new HashMap<String, FuncSpecCommand>();
+ HashMap<String, FuncSpecCommand> specSqlMap = new HashMap<String, FuncSpecCommand>();
+ HashMap<String, HplsqlParser.Create_function_stmtContext> userMap = new HashMap<String, HplsqlParser.Create_function_stmtContext>();
+ HashMap<String, HplsqlParser.Create_procedure_stmtContext> procMap = new HashMap<String, HplsqlParser.Create_procedure_stmtContext>();
+ boolean trace = false;
+
+ public Function(Exec e) {
+ exec = e;
+ trace = exec.getTrace();
+ }
+
+ /**
+ * Register functions
+ */
+ public void register(Function f) {
+ }
+
+ /**
+ * Execute a function
+ */
+ public void exec(String name, HplsqlParser.Expr_func_paramsContext ctx) {
+ if (execUser(ctx, name)) {
+ return;
+ }
+ if (name.indexOf(".") != -1) { // Name can be qualified and spaces are allowed between parts
+ String[] parts = name.split("\\.");
+ StringBuilder str = new StringBuilder();
+ for (int i = 0; i < parts.length; i++) {
+ if (i > 0) {
+ str.append(".");
+ }
+ str.append(parts[i].trim());
+ }
+ name = str.toString();
+ }
+ if (trace && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
+ trace(ctx, "FUNC " + name);
+ }
+ FuncCommand func = map.get(name);
+ if (func != null) {
+ func.run(ctx);
+ }
+ else {
+ evalNull();
+ }
+ }
+
+ /**
+ * User-defined function in a SQL query
+ */
+ public void execSql(String name, HplsqlParser.Expr_func_paramsContext ctx) {
+ if (execUserSql(ctx, name)) {
+ return;
+ }
+ StringBuilder sql = new StringBuilder();
+ sql.append(name);
+ sql.append("(");
+ int cnt = ctx.expr().size();
+ for (int i = 0; i < cnt; i++) {
+ sql.append(evalPop(ctx.expr(i)));
+ if (i + 1 < cnt) {
+ sql.append(", ");
+ }
+ }
+ sql.append(")");
+ exec.stackPush(sql);
+ }
+
+ /**
+ * Aggregate or window function in a SQL query
+ */
+ public void execAggWindowSql(HplsqlParser.Expr_agg_window_funcContext ctx) {
+ exec.stackPush(exec.getFormattedText(ctx));
+ }
+
+ /**
+ * Execute a user-defined function
+ */
+ public boolean execUser(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+ HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
+ if (userCtx == null) {
+ return false;
+ }
+ if (trace) {
+ trace(ctx, "EXEC FUNCTION " + name);
+ }
+ exec.enterScope(Scope.Type.ROUTINE);
+ setCallParameters(ctx, userCtx.create_routine_params(), null);
+ visit(userCtx.single_block_stmt());
+ exec.leaveScope();
+ return true;
+ }
+
+ /**
+ * Execute a HPL/SQL user-defined function in a query
+ */
+ public boolean execUserSql(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+ HplsqlParser.Create_function_stmtContext userCtx = userMap.get(name.toUpperCase());
+ if (userCtx == null) {
+ return false;
+ }
+ StringBuilder sql = new StringBuilder();
+ sql.append("hplsql('");
+ sql.append(name);
+ sql.append("(");
+ int cnt = ctx.expr().size();
+ for (int i = 0; i < cnt; i++) {
+ sql.append(":" + (i + 1));
+ if (i + 1 < cnt) {
+ sql.append(", ");
+ }
+ }
+ sql.append(")'");
+ if (cnt > 0) {
+ sql.append(", ");
+ }
+ for (int i = 0; i < cnt; i++) {
+ sql.append(evalPop(ctx.expr(i)));
+ if (i + 1 < cnt) {
+ sql.append(", ");
+ }
+ }
+ sql.append(")");
+ exec.stackPush(sql);
+ exec.registerUdf();
+ return true;
+ }
+
+ /**
+ * Execute a stored procedure as the entry point of the script (defined by -main option)
+ */
+ public boolean execProc(String name) {
+ if (trace) {
+ trace("EXEC PROCEDURE " + name);
+ }
+ HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());
+ if (procCtx == null) {
+ trace("Procedure not found");
+ return false;
+ }
+ exec.enterScope(Scope.Type.ROUTINE);
+ setCallParameters(procCtx.create_routine_params());
+ visit(procCtx.single_block_stmt());
+ exec.leaveScope();
+ return true;
+ }
+
+ /**
+ * Execute a stored procedure using CALL or EXEC statement passing parameters
+ */
+ public boolean execProc(HplsqlParser.Expr_func_paramsContext ctx, String name) {
+ if (trace) {
+ trace(ctx, "EXEC PROCEDURE " + name);
+ }
+ HplsqlParser.Create_procedure_stmtContext procCtx = procMap.get(name.toUpperCase());
+ if (procCtx == null) {
+ trace(ctx, "Procedure not found");
+ return false;
+ }
+ HashMap<String, Var> out = new HashMap<String, Var>();
+ exec.enterScope(Scope.Type.ROUTINE);
+ setCallParameters(ctx, procCtx.create_routine_params(), out);
+ visit(procCtx.single_block_stmt());
+ exec.leaveScope();
+ for (Map.Entry<String, Var> i : out.entrySet()) { // Set OUT parameters
+ exec.setVariable(i.getKey(), i.getValue());
+ }
+ return true;
+ }
+
+ /**
+ * Set parameters for user-defined function call
+ */
+ void setCallParameters(HplsqlParser.Expr_func_paramsContext actual,
+ HplsqlParser.Create_routine_paramsContext formal,
+ HashMap<String, Var> out) {
+ int actualCnt = actual.expr().size();
+ int formalCnt = formal.create_routine_param_item().size();
+ for (int i = 0; i < actualCnt; i++) {
+ if (i >= formalCnt) {
+ break;
+ }
+ HplsqlParser.ExprContext a = actual.expr(i);
+ HplsqlParser.Create_routine_param_itemContext p = formal.create_routine_param_item(i);
+ String name = p.ident().getText();
+ String type = p.dtype().getText();
+ String len = null;
+ String scale = null;
+ if (p.dtype_len() != null) {
+ len = p.dtype_len().L_INT(0).getText();
+ if (p.dtype_len().L_INT(1) != null) {
+ scale = p.dtype_len().L_INT(1).getText();
+ }
+ }
+ Var value = evalPop(a);
+ Var var = setCallParameter(name, type, len, scale, value);
+ if (trace) {
+ trace(actual, "SET PARAM " + name + " = " + var.toString());
+ }
+ if (out != null && a.expr_atom() != null && a.expr_atom().ident() != null &&
+ (p.T_OUT() != null || p.T_INOUT() != null)) {
+ String actualName = a.expr_atom().ident().getText();
+ if (actualName != null) {
+ out.put(actualName, var);
+ }
+ }
+ }
+ }
+
+ /**
+ * Set parameters for entry-point call (Main procedure defined by -main option)
+ */
+ void setCallParameters(HplsqlParser.Create_routine_paramsContext ctx) {
+ int cnt = ctx.create_routine_param_item().size();
+ for (int i = 0; i < cnt; i++) {
+ HplsqlParser.Create_routine_param_itemContext p = ctx.create_routine_param_item(i);
+ String name = p.ident().getText();
+ String type = p.dtype().getText();
+ String len = null;
+ String scale = null;
+ if (p.dtype_len() != null) {
+ len = p.dtype_len().L_INT(0).getText();
+ if (p.dtype_len().L_INT(1) != null) {
+ scale = p.dtype_len().L_INT(1).getText();
+ }
+ }
+ Var value = exec.findVariable(name);
+ Var var = setCallParameter(name, type, len, scale, value);
+ if (trace) {
+ trace(ctx, "SET PARAM " + name + " = " + var.toString());
+ }
+ }
+ }
+
+ /**
+ * Create a function or procedure parameter and set its value
+ */
+ Var setCallParameter(String name, String type, String len, String scale, Var value) {
+ Var var = new Var(name, type, len, scale, null);
+ var.cast(value);
+ exec.addVariable(var);
+ return var;
+ }
+
+ /**
+ * Add a user-defined function
+ */
+ public void addUserFunction(HplsqlParser.Create_function_stmtContext ctx) {
+ String name = ctx.ident().getText();
+ if (trace) {
+ trace(ctx, "CREATE FUNCTION " + name);
+ }
+ userMap.put(name.toUpperCase(), ctx);
+ }
+
+ /**
+ * Add a user-defined procedure
+ */
+ public void addUserProcedure(HplsqlParser.Create_procedure_stmtContext ctx) {
+ String name = ctx.ident(0).getText();
+ if (trace) {
+ trace(ctx, "CREATE PROCEDURE " + name);
+ }
+ procMap.put(name.toUpperCase(), ctx);
+ }
+
+ /**
+ * Execute a special function
+ */
+ public void specExec(HplsqlParser.Expr_spec_funcContext ctx) {
+ String name = ctx.start.getText().toUpperCase();
+ if (trace && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
+ trace(ctx, "FUNC " + name);
+ }
+ FuncSpecCommand func = specMap.get(name);
+ if (func != null) {
+ func.run(ctx);
+ }
+ else if(ctx.T_MAX_PART_STRING() != null) {
+ execMaxPartString(ctx);
+ } else if(ctx.T_MIN_PART_STRING() != null) {
+ execMinPartString(ctx);
+ } else if(ctx.T_MAX_PART_INT() != null) {
+ execMaxPartInt(ctx);
+ } else if(ctx.T_MIN_PART_INT() != null) {
+ execMinPartInt(ctx);
+ } else if(ctx.T_MAX_PART_DATE() != null) {
+ execMaxPartDate(ctx);
+ } else if(ctx.T_MIN_PART_DATE() != null) {
+ execMinPartDate(ctx);
+ } else if(ctx.T_PART_LOC() != null) {
+ execPartLoc(ctx);
+ } else if(ctx.T_SYSDATE() != null) {
+ execCurrentTimestamp(ctx, 0);
+ } else {
+ evalNull();
+ }
+ }
+
+ /**
+ * Execute a special function in executable SQL statement
+ */
+ public void specExecSql(HplsqlParser.Expr_spec_funcContext ctx) {
+ String name = ctx.start.getText().toUpperCase();
+ if (trace && ctx.parent.parent instanceof HplsqlParser.Expr_stmtContext) {
+ trace(ctx, "FUNC " + name);
+ }
+ FuncSpecCommand func = specSqlMap.get(name);
+ if (func != null) {
+ func.run(ctx);
+ }
+ else {
+ exec.stackPush(exec.getFormattedText(ctx));
+ }
+ }
+
+ /**
+ * Get the current date
+ */
+ public void execCurrentDate(HplsqlParser.Expr_spec_funcContext ctx) {
+ if(trace) {
+ trace(ctx, "CURRENT_DATE");
+ }
+ SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+ String s = f.format(Calendar.getInstance().getTime());
+ exec.stackPush(new Var(Var.Type.DATE, Utils.toDate(s)));
+ }
+
+ /**
+ * Get the current date and time
+ */
+ public void execCurrentTimestamp(HplsqlParser.Expr_spec_funcContext ctx, int defPrecision) {
+ trace(ctx, "CURRENT_TIMESTAMP");
+ int precision = evalPop(ctx.expr(0), defPrecision).intValue();
+ String format = "yyyy-MM-dd HH:mm:ss";
+ if(precision > 0 && precision <= 3) {
+ format += "." + StringUtils.repeat("S", precision);
+ }
+ SimpleDateFormat f = new SimpleDateFormat(format);
+ String s = f.format(Calendar.getInstance(TimeZone.getDefault()).getTime());
+ exec.stackPush(new Var(Utils.toTimestamp(s), precision));
+ }
+
+ /**
+ * Execute MAX_PART_STRING function
+ */
+ public void execMaxPartString(HplsqlParser.Expr_spec_funcContext ctx) {
+ if(trace) {
+ trace(ctx, "MAX_PART_STRING");
+ }
+ execMinMaxPart(ctx, Var.Type.STRING, true /*max*/);
+ }
+
+ /**
+ * Execute MIN_PART_STRING function
+ */
+ public void execMinPartString(HplsqlParser.Expr_spec_funcContext ctx) {
+ if(trace) {
+ trace(ctx, "MIN_PART_STRING");
+ }
+ execMinMaxPart(ctx, Var.Type.STRING, false /*max*/);
+ }
+
+ /**
+ * Execute MAX_PART_INT function
+ */
+ public void execMaxPartInt(HplsqlParser.Expr_spec_funcContext ctx) {
+ if(trace) {
+ trace(ctx, "MAX_PART_INT");
+ }
+ execMinMaxPart(ctx, Var.Type.BIGINT, true /*max*/);
+ }
+
+ /**
+ * Execute MIN_PART_INT function
+ */
+ public void execMinPartInt(HplsqlParser.Expr_spec_funcContext ctx) {
+ if(trace) {
+ trace(ctx, "MIN_PART_INT");
+ }
+ execMinMaxPart(ctx, Var.Type.BIGINT, false /*max*/);
+ }
+
+ /**
+ * Execute MAX_PART_DATE function
+ */
+ public void execMaxPartDate(HplsqlParser.Expr_spec_funcContext ctx) {
+ if(trace) {
+ trace(ctx, "MAX_PART_DATE");
+ }
+ execMinMaxPart(ctx, Var.Type.DATE, true /*max*/);
+ }
+
+ /**
+ * Execute MIN_PART_DATE function
+ */
+ public void execMinPartDate(HplsqlParser.Expr_spec_funcContext ctx) {
+ if(trace) {
+ trace(ctx, "MIN_PART_DATE");
+ }
+ execMinMaxPart(ctx, Var.Type.DATE, false /*max*/);
+ }
+
+ /**
+ * Execute MIN or MAX partition function
+ */
+ public void execMinMaxPart(HplsqlParser.Expr_spec_funcContext ctx, Var.Type type, boolean max) {
+ String tabname = evalPop(ctx.expr(0)).toString();
+ String sql = "SHOW PARTITIONS " + tabname;
+ String colname = null;
+ int colnum = -1;
+ int exprnum = ctx.expr().size();
+ // Column name
+ if (ctx.expr(1) != null) {
+ colname = evalPop(ctx.expr(1)).toString();
+ } else {
+ colnum = 0;
+ }
+ // Partition filter
+ if (exprnum >= 4) {
+ sql += " PARTITION (";
+ int i = 2;
+ while (i + 1 < exprnum) {
+ String fcol = evalPop(ctx.expr(i)).toString();
+ String fval = evalPop(ctx.expr(i+1)).toSqlString();
+ if (i > 2) {
+ sql += ", ";
+ }
+ sql += fcol + "=" + fval;
+ i += 2;
+ }
+ sql += ")";
+ }
+ if (trace) {
+ trace(ctx, "Query: " + sql);
+ }
+ if (exec.getOffline()) {
+ evalNull();
+ return;
+ }
+ Query query = exec.executeQuery(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ evalNullClose(query, exec.conf.defaultConnection);
+ return;
+ }
+ ResultSet rs = query.getResultSet();
+ try {
+ String resultString = null;
+ Long resultInt = null;
+ Date resultDate = null;
+ while (rs.next()) {
+ String[] parts = rs.getString(1).split("/");
+ // Find partition column by name
+ if (colnum == -1) {
+ for (int i = 0; i < parts.length; i++) {
+ String[] name = parts[i].split("=");
+ if (name[0].equalsIgnoreCase(colname)) {
+ colnum = i;
+ break;
+ }
+ }
+ // No partition column with the specified name exists
+ if (colnum == -1) {
+ evalNullClose(query, exec.conf.defaultConnection);
+ return;
+ }
+ }
+ String[] pair = parts[colnum].split("=");
+ if (type == Var.Type.STRING) {
+ resultString = Utils.minMaxString(resultString, pair[1], max);
+ }
+ else if (type == Var.Type.BIGINT) {
+ resultInt = Utils.minMaxInt(resultInt, pair[1], max);
+ }
+ else if (type == Var.Type.DATE) {
+ resultDate = Utils.minMaxDate(resultDate, pair[1], max);
+ }
+ }
+ if (resultString != null) {
+ evalString(resultString);
+ }
+ else if (resultInt != null) {
+ evalInt(resultInt);
+ }
+ else if (resultDate != null) {
+ evalDate(resultDate);
+ }
+ else {
+ evalNull();
+ }
+ } catch (SQLException e) {}
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ }
+
+ /**
+ * Execute PART_LOC function
+ */
+ public void execPartLoc(HplsqlParser.Expr_spec_funcContext ctx) {
+ String tabname = evalPop(ctx.expr(0)).toString();
+ String sql = "DESCRIBE EXTENDED " + tabname;
+ int exprnum = ctx.expr().size();
+ boolean hostname = false;
+ // Partition filter
+ if (exprnum > 1) {
+ sql += " PARTITION (";
+ int i = 1;
+ while (i + 1 < exprnum) {
+ String col = evalPop(ctx.expr(i)).toString();
+ String val = evalPop(ctx.expr(i+1)).toSqlString();
+ if (i > 2) {
+ sql += ", ";
+ }
+ sql += col + "=" + val;
+ i += 2;
+ }
+ sql += ")";
+ }
+ // With host name
+ if (exprnum % 2 == 0 && evalPop(ctx.expr(exprnum - 1)).intValue() == 1) {
+ hostname = true;
+ }
+ if (trace) {
+ trace(ctx, "Query: " + sql);
+ }
+ if (exec.getOffline()) {
+ evalNull();
+ return;
+ }
+ Query query = exec.executeQuery(ctx, sql, exec.conf.defaultConnection);
+ if (query.error()) {
+ evalNullClose(query, exec.conf.defaultConnection);
+ return;
+ }
+ String result = null;
+ ResultSet rs = query.getResultSet();
+ try {
+ while (rs.next()) {
+ if (rs.getString(1).startsWith("Detailed Partition Information")) {
+ Matcher m = Pattern.compile(".*, location:(.*?),.*").matcher(rs.getString(2));
+ if (m.find()) {
+ result = m.group(1);
+ }
+ }
+ }
+ } catch (SQLException e) {}
+ if (result != null) {
+ // Remove the host name
+ if (!hostname) {
+ Matcher m = Pattern.compile(".*://.*?(/.*)").matcher(result);
+ if (m.find()) {
+ result = m.group(1);
+ }
+ }
+ evalString(result);
+ }
+ else {
+ evalNull();
+ }
+ exec.closeQuery(query, exec.conf.defaultConnection);
+ }
+
+ /**
+ * Evaluate the expression and push the value to the stack
+ */
+ void eval(ParserRuleContext ctx) {
+ exec.visit(ctx);
+ }
+
+ /**
+ * Evaluate the expression to the specified variable
+ */
+ void evalVar(Var var) {
+ exec.stackPush(var);
+ }
+
+ /**
+ * Evaluate the expression to NULL
+ */
+ void evalNull() {
+ exec.stackPush(Var.Null);
+ }
+
+ /**
+ * Evaluate the expression to specified String value
+ */
+ void evalString(String string) {
+ exec.stackPush(new Var(string));
+ }
+
+ void evalString(StringBuilder string) {
+ evalString(string.toString());
+ }
+
+ /**
+ * Evaluate the expression to specified Int value
+ */
+ void evalInt(Long i) {
+ exec.stackPush(new Var(i));
+ }
+
+ /**
+ * Evaluate the expression to specified Date value
+ */
+ void evalDate(Date date) {
+ exec.stackPush(new Var(Var.Type.DATE, date));
+ }
+
+ /**
+ * Evaluate the expression to NULL and close the query
+ */
+ void evalNullClose(Query query, String conn) {
+ exec.stackPush(Var.Null);
+ exec.closeQuery(query, conn);
+ if(trace) {
+ query.printStackTrace();
+ }
+ }
+
+ /**
+ * Evaluate the expression and pop value from the stack
+ */
+ Var evalPop(ParserRuleContext ctx) {
+ exec.visit(ctx);
+ return exec.stackPop();
+ }
+
+ Var evalPop(ParserRuleContext ctx, int value) {
+ if (ctx != null) {
+ return evalPop(ctx);
+ }
+ return new Var(new Long(value));
+ }
+
+ /**
+ * Execute rules
+ */
+ Integer visit(ParserRuleContext ctx) {
+ return exec.visit(ctx);
+ }
+
+ /**
+ * Execute children rules
+ */
+ Integer visitChildren(ParserRuleContext ctx) {
+ return exec.visitChildren(ctx);
+ }
+
+ /**
+ * Trace information
+ */
+ public void trace(ParserRuleContext ctx, String message) {
+ if (trace) {
+ exec.trace(ctx, message);
+ }
+ }
+
+ public void trace(String message) {
+ trace(null, message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
new file mode 100644
index 0000000..926eeeb
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionDatetime.java
@@ -0,0 +1,151 @@
+/**
+ * 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.hive.hplsql.functions;
+
+import java.sql.Timestamp;
+import java.text.SimpleDateFormat;
+import java.util.Calendar;
+import java.util.TimeZone;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hive.hplsql.*;
+
+public class FunctionDatetime extends Function {
+ public FunctionDatetime(Exec e) {
+ super(e);
+ }
+
+ /**
+ * Register functions
+ */
+ @Override
+ public void register(Function f) {
+ f.map.put("DATE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { date(ctx); }});
+ f.map.put("TIMESTAMP_ISO", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { timestampIso(ctx); }});
+ f.map.put("TO_TIMESTAMP", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { toTimestamp(ctx); }});
+
+ f.specMap.put("CURRENT_DATE", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentDate(ctx); }});
+ f.specMap.put("CURRENT_TIMESTAMP", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentTimestamp(ctx); }});
+
+ f.specSqlMap.put("CURRENT_DATE", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentDateSql(ctx); }});
+ f.specSqlMap.put("CURRENT_TIMESTAMP", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentTimestampSql(ctx); }});
+ }
+
+ /**
+ * CURRENT_DATE
+ */
+ public void currentDate(HplsqlParser.Expr_spec_funcContext ctx) {
+ evalVar(currentDate());
+ }
+
+ public static Var currentDate() {
+ SimpleDateFormat f = new SimpleDateFormat("yyyy-MM-dd");
+ String s = f.format(Calendar.getInstance().getTime());
+ return new Var(Var.Type.DATE, Utils.toDate(s));
+ }
+
+ /**
+ * CURRENT_DATE in executable SQL statement
+ */
+ public void currentDateSql(HplsqlParser.Expr_spec_funcContext ctx) {
+ if (exec.getConnectionType() == Conn.Type.HIVE) {
+ evalString("TO_DATE(FROM_UNIXTIME(UNIX_TIMESTAMP()))");
+ }
+ else {
+ evalString(exec.getFormattedText(ctx));
+ }
+ }
+
+ /**
+ * CURRENT_TIMESTAMP
+ */
+ public void currentTimestamp(HplsqlParser.Expr_spec_funcContext ctx) {
+ int precision = evalPop(ctx.expr(0), 3).intValue();
+ evalVar(currentTimestamp(precision));
+ }
+
+ public static Var currentTimestamp(int precision) {
+ String format = "yyyy-MM-dd HH:mm:ss";
+ if (precision > 0 && precision <= 3) {
+ format += "." + StringUtils.repeat("S", precision);
+ }
+ SimpleDateFormat f = new SimpleDateFormat(format);
+ String s = f.format(Calendar.getInstance(TimeZone.getDefault()).getTime());
+ return new Var(Utils.toTimestamp(s), precision);
+ }
+
+ /**
+ * CURRENT_TIMESTAMP in executable SQL statement
+ */
+ public void currentTimestampSql(HplsqlParser.Expr_spec_funcContext ctx) {
+ if (exec.getConnectionType() == Conn.Type.HIVE) {
+ evalString("FROM_UNIXTIME(UNIX_TIMESTAMP())");
+ }
+ else {
+ evalString(exec.getFormattedText(ctx));
+ }
+ }
+
+ /**
+ * DATE function
+ */
+ void date(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() != 1) {
+ evalNull();
+ return;
+ }
+ Var var = new Var(Var.Type.DATE);
+ var.cast(evalPop(ctx.expr(0)));
+ evalVar(var);
+ }
+
+ /**
+ * TIMESTAMP_ISO function
+ */
+ void timestampIso(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() != 1) {
+ evalNull();
+ return;
+ }
+ Var var = new Var(Var.Type.TIMESTAMP);
+ var.cast(evalPop(ctx.expr(0)));
+ evalVar(var);
+ }
+
+ /**
+ * TO_TIMESTAMP function
+ */
+ void toTimestamp(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() != 2) {
+ evalNull();
+ return;
+ }
+ String value = evalPop(ctx.expr(0)).toString();
+ String sqlFormat = evalPop(ctx.expr(1)).toString();
+ String format = Utils.convertSqlDatetimeFormat(sqlFormat);
+ try {
+ long timeInMs = new SimpleDateFormat(format).parse(value).getTime();
+ evalVar(new Var(Var.Type.TIMESTAMP, new Timestamp(timeInMs)));
+ }
+ catch (Exception e) {
+ exec.signal(e);
+ evalNull();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
new file mode 100644
index 0000000..8d2456c
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionMisc.java
@@ -0,0 +1,188 @@
+/**
+ * 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.hive.hplsql.functions;
+
+import org.apache.hive.hplsql.*;
+
+public class FunctionMisc extends Function {
+ public FunctionMisc(Exec e) {
+ super(e);
+ }
+
+ /**
+ * Register functions
+ */
+ @Override
+ public void register(Function f) {
+ f.map.put("COALESCE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl(ctx); }});
+ f.map.put("DECODE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { decode(ctx); }});
+ f.map.put("NVL", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl(ctx); }});
+ f.map.put("NVL2", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { nvl2(ctx); }});
+
+ f.specMap.put("ACTIVITY_COUNT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { activityCount(ctx); }});
+ f.specMap.put("CAST", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { cast(ctx); }});
+ f.specMap.put("CURRENT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { current(ctx); }});
+ f.specMap.put("CURRENT_USER", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentUser(ctx); }});
+ f.specMap.put("USER", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentUser(ctx); }});
+
+ f.specSqlMap.put("CURRENT", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { currentSql(ctx); }});
+ }
+
+ /**
+ * ACTIVITY_COUNT function (built-in variable)
+ */
+ void activityCount(HplsqlParser.Expr_spec_funcContext ctx) {
+ evalInt(new Long(exec.getRowCount()));
+ }
+
+ /**
+ * CAST function
+ */
+ void cast(HplsqlParser.Expr_spec_funcContext ctx) {
+ if (ctx.expr().size() != 1) {
+ evalNull();
+ return;
+ }
+ String type = ctx.dtype().getText();
+ String len = null;
+ String scale = null;
+ if (ctx.dtype_len() != null) {
+ len = ctx.dtype_len().L_INT(0).getText();
+ if (ctx.dtype_len().L_INT(1) != null) {
+ scale = ctx.dtype_len().L_INT(1).getText();
+ }
+ }
+ Var var = new Var(null, type, len, scale, null);
+ var.cast(evalPop(ctx.expr(0)));
+ evalVar(var);
+ }
+
+ /**
+ * CURRENT <VALUE> function
+ */
+ void current(HplsqlParser.Expr_spec_funcContext ctx) {
+ if (ctx.T_DATE() != null) {
+ evalVar(FunctionDatetime.currentDate());
+ }
+ else if (ctx.T_TIMESTAMP() != null) {
+ int precision = evalPop(ctx.expr(0), 3).intValue();
+ evalVar(FunctionDatetime.currentTimestamp(precision));
+ }
+ else if (ctx.T_USER() != null) {
+ evalVar(FunctionMisc.currentUser());
+ }
+ else {
+ evalNull();
+ }
+ }
+
+ /**
+ * CURRENT <VALUE> function in executable SQL statement
+ */
+ void currentSql(HplsqlParser.Expr_spec_funcContext ctx) {
+ if (ctx.T_DATE() != null) {
+ if (exec.getConnectionType() == Conn.Type.HIVE) {
+ evalString("TO_DATE(FROM_UNIXTIME(UNIX_TIMESTAMP()))");
+ }
+ else {
+ evalString("CURRENT_DATE");
+ }
+ }
+ else if (ctx.T_TIMESTAMP() != null) {
+ if (exec.getConnectionType() == Conn.Type.HIVE) {
+ evalString("FROM_UNIXTIME(UNIX_TIMESTAMP())");
+ }
+ else {
+ evalString("CURRENT_TIMESTAMP");
+ }
+ }
+ else {
+ evalString(exec.getFormattedText(ctx));
+ }
+ }
+
+ /**
+ * CURRENT_USER function
+ */
+ void currentUser(HplsqlParser.Expr_spec_funcContext ctx) {
+ evalVar(currentUser());
+ }
+
+ public static Var currentUser() {
+ return new Var(System.getProperty("user.name"));
+ }
+
+ /**
+ * DECODE function
+ */
+ void decode(HplsqlParser.Expr_func_paramsContext ctx) {
+ int cnt = ctx.expr().size();
+ if (cnt < 3) {
+ evalNull();
+ return;
+ }
+ Var value = evalPop(ctx.expr(0));
+ int i = 1;
+ while (i + 1 < cnt) {
+ Var when = evalPop(ctx.expr(i));
+ if ((value.isNull() && when.isNull()) || value.equals(when)) {
+ eval(ctx.expr(i + 1));
+ return;
+ }
+ i += 2;
+ }
+ if (i < cnt) { // ELSE expression
+ eval(ctx.expr(i));
+ }
+ else {
+ evalNull();
+ }
+ }
+
+ /**
+ * NVL function - Return first non-NULL expression
+ */
+ void nvl(HplsqlParser.Expr_func_paramsContext ctx) {
+ for (int i=0; i < ctx.expr().size(); i++) {
+ Var v = evalPop(ctx.expr(i));
+ if (v.type != Var.Type.NULL) {
+ exec.stackPush(v);
+ return;
+ }
+ }
+ evalNull();
+ }
+
+ /**
+ * NVL2 function - If expr1 is not NULL return expr2, otherwise expr3
+ */
+ void nvl2(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() == 3) {
+ if (!evalPop(ctx.expr(0)).isNull()) {
+ eval(ctx.expr(1));
+ }
+ else {
+ eval(ctx.expr(2));
+ }
+ }
+ else {
+ evalNull();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java
new file mode 100644
index 0000000..fec891a
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionOra.java
@@ -0,0 +1,231 @@
+/**
+ * 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.hive.hplsql.functions;
+
+import java.io.IOException;
+import java.io.EOFException;
+
+import org.apache.hive.hplsql.*;
+
+public class FunctionOra extends Function {
+ public FunctionOra(Exec e) {
+ super(e);
+ }
+
+ /**
+ * Register functions
+ */
+ @Override
+ public void register(Function f) {
+ f.map.put("DBMS_OUTPUT.PUT_LINE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) {
+ execDbmsOutputPutLine(ctx); }});
+ f.map.put("UTL_FILE.FOPEN", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) {
+ execUtlFileFopen(ctx); }});
+ f.map.put("UTL_FILE.GET_LINE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) {
+ execUtlFileGetLine(ctx); }});
+ f.map.put("UTL_FILE.PUT_LINE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) {
+ execUtlFilePutLine(ctx); }});
+ f.map.put("UTL_FILE.PUT", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) {
+ execUtlFilePut(ctx); }});
+ f.map.put("UTL_FILE.FCLOSE", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) {
+ execUtlFileFclose(ctx); }});
+ }
+
+ /**
+ * Print a text message
+ */
+ void execDbmsOutputPutLine(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() > 0) {
+ visit(ctx.expr(0));
+ System.out.println(exec.stackPop().toString());
+ }
+ }
+
+ /**
+ * Execute UTL_FILE.FOPEN function
+ */
+ public void execUtlFileFopen(HplsqlParser.Expr_func_paramsContext ctx) {
+ String dir = "";
+ String name = "";
+ boolean write = true;
+ boolean overwrite = false;
+ int cnt = ctx.expr().size();
+ // Directory
+ if (cnt > 0) {
+ dir = evalPop(ctx.expr(0)).toString();
+ }
+ // File name
+ if (cnt > 1) {
+ name = evalPop(ctx.expr(1)).toString();
+ }
+ // Mode
+ if (cnt >= 2) {
+ String mode = evalPop(ctx.expr(2)).toString();
+ if (mode.equalsIgnoreCase("r")) {
+ write = false;
+ }
+ else if (mode.equalsIgnoreCase("w")) {
+ write = true;
+ overwrite = true;
+ }
+ }
+ File file = new File();
+ if (write) {
+ file.create(dir, name, overwrite);
+ }
+ else {
+ file.open(dir, name);
+ }
+ exec.stackPush(new Var(Var.Type.FILE, file));
+ }
+
+ /**
+ * Read a text line from an open file
+ */
+ void execUtlFileGetLine(HplsqlParser.Expr_func_paramsContext ctx) {
+ int cnt = ctx.expr().size();
+ Var file = null;
+ Var str = null;
+ StringBuilder out = new StringBuilder();
+
+ // File handle
+ if(cnt > 0) {
+ visit(ctx.expr(0));
+ file = exec.stackPop();
+ }
+ // String variable
+ if(cnt > 1) {
+ visit(ctx.expr(1));
+ str = exec.stackPop();
+ }
+
+ if(file != null && file.type == Var.Type.FILE) {
+ File f = (File)file.value;
+
+ if(trace) {
+ trace(ctx, "File: " + f.toString());
+ }
+
+ try {
+ while(true) {
+ char c = f.readChar();
+ if(c == '\n') {
+ break;
+ }
+ out.append(c);
+ }
+ } catch (IOException e) {
+ if(!(e instanceof EOFException)) {
+ out.setLength(0);
+ }
+ }
+
+ // Set the new value to the output string variable
+ if(str != null) {
+ str.setValue(out.toString());
+
+ if(trace) {
+ trace(ctx, "OUT " + str.getName() + " = " + str.toString());
+ }
+ }
+ }
+ else if(trace) {
+ trace(ctx, "Variable of FILE type not found");
+ }
+ }
+
+ /**
+ * Execute UTL_FILE.PUT_LINE function
+ */
+ public void execUtlFilePutLine(HplsqlParser.Expr_func_paramsContext ctx) {
+ execUtlFilePut(ctx, true /*newline*/);
+ }
+
+ /**
+ * Execute UTL_FILE.PUT function
+ */
+ public void execUtlFilePut(HplsqlParser.Expr_func_paramsContext ctx) {
+ execUtlFilePut(ctx, false /*newline*/);
+ }
+
+ /**
+ * Write a string to file
+ */
+ void execUtlFilePut(HplsqlParser.Expr_func_paramsContext ctx, boolean newline) {
+ int cnt = ctx.expr().size();
+ Var file = null;
+ String str = "";
+
+ // File handle
+ if(cnt > 0) {
+ visit(ctx.expr(0));
+ file = exec.stackPop();
+ }
+ // Text string
+ if(cnt > 1) {
+ visit(ctx.expr(1));
+ str = exec.stackPop().toString();
+ }
+
+ if(file != null && file.type == Var.Type.FILE) {
+ File f = (File)file.value;
+
+ if(trace) {
+ trace(ctx, "File: " + f.toString());
+ }
+
+ f.writeString(str);
+
+ if(newline) {
+ f.writeString("\n");
+ }
+ }
+ else if(trace) {
+ trace(ctx, "Variable of FILE type not found");
+ }
+ }
+
+ /**
+ * Execute UTL_FILE.FCLOSE function
+ */
+ void execUtlFileFclose(HplsqlParser.Expr_func_paramsContext ctx) {
+ int cnt = ctx.expr().size();
+ Var file = null;
+
+ // File handle
+ if(cnt > 0) {
+ visit(ctx.expr(0));
+ file = exec.stackPop();
+ }
+
+ if(file != null && file.type == Var.Type.FILE) {
+ File f = (File)file.value;
+
+ if(trace) {
+ trace(ctx, "File: " + f.toString());
+ }
+
+ f.close();
+ file.removeValue();
+ }
+ else if(trace) {
+ trace(ctx, "Variable of FILE type not found");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java
----------------------------------------------------------------------
diff --git a/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java
new file mode 100644
index 0000000..610ff60
--- /dev/null
+++ b/hplsql/src/main/java/org/apache/hive/hplsql/functions/FunctionString.java
@@ -0,0 +1,276 @@
+/**
+ * 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.hive.hplsql.functions;
+
+import org.apache.hive.hplsql.*;
+
+public class FunctionString extends Function {
+ public FunctionString(Exec e) {
+ super(e);
+ }
+
+ /**
+ * Register functions
+ */
+ @Override
+ public void register(Function f) {
+ f.map.put("CONCAT", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { concat(ctx); }});
+ f.map.put("CHAR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { char_(ctx); }});
+ f.map.put("INSTR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { instr(ctx); }});
+ f.map.put("LEN", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { len(ctx); }});
+ f.map.put("LENGTH", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { length(ctx); }});
+ f.map.put("LOWER", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { lower(ctx); }});
+ f.map.put("SUBSTR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { substr(ctx); }});
+ f.map.put("SUBSTRING", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { substr(ctx); }});
+ f.map.put("TO_CHAR", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { toChar(ctx); }});
+ f.map.put("UPPER", new FuncCommand() { public void run(HplsqlParser.Expr_func_paramsContext ctx) { upper(ctx); }});
+
+ f.specMap.put("SUBSTRING", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { substring(ctx); }});
+ f.specMap.put("TRIM", new FuncSpecCommand() { public void run(HplsqlParser.Expr_spec_funcContext ctx) { trim(ctx); }});
+ }
+
+ /**
+ * CONCAT function
+ */
+ void concat(HplsqlParser.Expr_func_paramsContext ctx) {
+ StringBuilder val = new StringBuilder();
+ int cnt = ctx.expr().size();
+ boolean nulls = true;
+ for (int i = 0; i < cnt; i++) {
+ Var c = evalPop(ctx.expr(i));
+ if (!c.isNull()) {
+ val.append(c.toString());
+ nulls = false;
+ }
+ }
+ if (nulls) {
+ evalNull();
+ }
+ else {
+ evalString(val);
+ }
+ }
+
+ /**
+ * CHAR function
+ */
+ void char_(HplsqlParser.Expr_func_paramsContext ctx) {
+ int cnt = ctx.expr().size();
+ if (cnt != 1) {
+ evalNull();
+ return;
+ }
+ String str = evalPop(ctx.expr(0)).toString();
+ evalString(str);
+ }
+
+ /**
+ * INSTR function
+ */
+ void instr(HplsqlParser.Expr_func_paramsContext ctx) {
+ int cnt = ctx.expr().size();
+ if (cnt < 2) {
+ evalNull();
+ return;
+ }
+ String str = evalPop(ctx.expr(0)).toString();
+ if (str == null) {
+ evalNull();
+ return;
+ }
+ else if(str.isEmpty()) {
+ evalInt(new Long(0));
+ return;
+ }
+ String substr = evalPop(ctx.expr(1)).toString();
+ int pos = 1;
+ int occur = 1;
+ int idx = 0;
+ if (cnt >= 3) {
+ pos = evalPop(ctx.expr(2)).intValue();
+ if (pos == 0) {
+ pos = 1;
+ }
+ }
+ if (cnt >= 4) {
+ occur = evalPop(ctx.expr(3)).intValue();
+ if (occur < 0) {
+ occur = 1;
+ }
+ }
+ for (int i = occur; i > 0; i--) {
+ if (pos > 0) {
+ idx = str.indexOf(substr, pos - 1);
+ }
+ else {
+ str = str.substring(0, str.length() - pos*(-1));
+ idx = str.lastIndexOf(substr);
+ }
+ if (idx == -1) {
+ idx = 0;
+ break;
+ }
+ else {
+ idx++;
+ }
+ if (i > 1) {
+ if (pos > 0) {
+ pos = idx + 1;
+ }
+ else {
+ pos = (str.length() - idx + 1) * (-1);
+ }
+ }
+ }
+ evalInt(new Long(idx));
+ }
+
+ /**
+ * LEN function (excluding trailing spaces)
+ */
+ void len(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() != 1) {
+ evalNull();
+ return;
+ }
+ int len = evalPop(ctx.expr(0)).toString().trim().length();
+ evalInt(new Long(len));
+ }
+
+ /**
+ * LENGTH function
+ */
+ void length(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() != 1) {
+ evalNull();
+ return;
+ }
+ int len = evalPop(ctx.expr(0)).toString().length();
+ evalInt(new Long(len));
+ }
+
+ /**
+ * LOWER function
+ */
+ void lower(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() != 1) {
+ evalNull();
+ return;
+ }
+ String str = evalPop(ctx.expr(0)).toString().toLowerCase();
+ evalString(str);
+ }
+
+ /**
+ * SUBSTR and SUBSTRING function
+ */
+ void substr(HplsqlParser.Expr_func_paramsContext ctx) {
+ int cnt = ctx.expr().size();
+ if (cnt < 2) {
+ evalNull();
+ return;
+ }
+ String str = evalPop(ctx.expr(0)).toString();
+ int start = evalPop(ctx.expr(1)).intValue();
+ int len = -1;
+ if (start == 0) {
+ start = 1;
+ }
+ if (cnt > 2) {
+ len = evalPop(ctx.expr(2)).intValue();
+ }
+ substr(str, start, len);
+ }
+
+ void substr(String str, int start, int len) {
+ if (str == null) {
+ evalNull();
+ return;
+ }
+ else if (str.isEmpty()) {
+ evalString(str);
+ return;
+ }
+ if (start == 0) {
+ start = 1;
+ }
+ if (len == -1) {
+ if (start > 0) {
+ evalString(str.substring(start - 1));
+ }
+ }
+ else {
+ evalString(str.substring(start - 1, start - 1 + len));
+ }
+ }
+
+ /**
+ * SUBSTRING FROM FOR function
+ */
+ void substring(HplsqlParser.Expr_spec_funcContext ctx) {
+ String str = evalPop(ctx.expr(0)).toString();
+ int start = evalPop(ctx.expr(1)).intValue();
+ int len = -1;
+ if (start == 0) {
+ start = 1;
+ }
+ if (ctx.T_FOR() != null) {
+ len = evalPop(ctx.expr(2)).intValue();
+ }
+ substr(str, start, len);
+ }
+
+ /**
+ * TRIM function
+ */
+ void trim(HplsqlParser.Expr_spec_funcContext ctx) {
+ int cnt = ctx.expr().size();
+ if (cnt != 1) {
+ evalNull();
+ return;
+ }
+ String str = evalPop(ctx.expr(0)).toString();
+ evalString(str.trim());
+ }
+
+ /**
+ * TO_CHAR function
+ */
+ void toChar(HplsqlParser.Expr_func_paramsContext ctx) {
+ int cnt = ctx.expr().size();
+ if (cnt != 1) {
+ evalNull();
+ return;
+ }
+ String str = evalPop(ctx.expr(0)).toString();
+ evalString(str);
+ }
+
+ /**
+ * UPPER function
+ */
+ void upper(HplsqlParser.Expr_func_paramsContext ctx) {
+ if (ctx.expr().size() != 1) {
+ evalNull();
+ return;
+ }
+ String str = evalPop(ctx.expr(0)).toString().toUpperCase();
+ evalString(str);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/052643cb/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1c9b02f..f84f3e9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,6 +39,7 @@
<module>contrib</module>
<module>hbase-handler</module>
<module>hcatalog</module>
+ <module>hplsql</module>
<module>hwi</module>
<module>jdbc</module>
<module>metastore</module>
[02/25] hive git commit: HIVE-9566: HiveServer2 fails to start with
NullPointerException (Na via Xuefu)
Posted by se...@apache.org.
HIVE-9566: HiveServer2 fails to start with NullPointerException (Na via Xuefu)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/97b4750c
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/97b4750c
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/97b4750c
Branch: refs/heads/llap
Commit: 97b4750c6314eea9025b426e4df73f795b601927
Parents: 8ed3377
Author: Xuefu Zhang <xz...@Cloudera.com>
Authored: Tue Jun 30 05:15:40 2015 -0700
Committer: Xuefu Zhang <xz...@Cloudera.com>
Committed: Tue Jun 30 05:15:40 2015 -0700
----------------------------------------------------------------------
.../src/java/org/apache/hive/service/server/HiveServer2.java | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/97b4750c/service/src/java/org/apache/hive/service/server/HiveServer2.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/server/HiveServer2.java b/service/src/java/org/apache/hive/service/server/HiveServer2.java
index 00ab75f..4a4be97 100644
--- a/service/src/java/org/apache/hive/service/server/HiveServer2.java
+++ b/service/src/java/org/apache/hive/service/server/HiveServer2.java
@@ -317,7 +317,7 @@ public class HiveServer2 extends CompositeService {
}
}
// Remove this server instance from ZooKeeper if dynamic service discovery is set
- if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) {
+ if (hiveConf != null && hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_SUPPORT_DYNAMIC_SERVICE_DISCOVERY)) {
try {
removeServerInstanceFromZooKeeper();
} catch (Exception e) {
@@ -326,7 +326,7 @@ public class HiveServer2 extends CompositeService {
}
// There should already be an instance of the session pool manager.
// If not, ignoring is fine while stopping HiveServer2.
- if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS)) {
+ if (hiveConf != null && hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_TEZ_INITIALIZE_DEFAULT_SESSIONS)) {
try {
TezSessionPoolManager.getInstance().stop();
} catch (Exception e) {
@@ -335,7 +335,7 @@ public class HiveServer2 extends CompositeService {
}
}
- if (hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {
+ if (hiveConf != null && hiveConf.getVar(ConfVars.HIVE_EXECUTION_ENGINE).equals("spark")) {
try {
SparkSessionManagerImpl.getInstance().shutdown();
} catch(Exception ex) {
[09/25] hive git commit: HIVE-11147 : MetaTool doesn't update FS root
location for partitions with space in name (Wei Zheng via Thejas Nair)
Posted by se...@apache.org.
HIVE-11147 : MetaTool doesn't update FS root location for partitions with space in name (Wei Zheng via Thejas Nair)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/edb7b889
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/edb7b889
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/edb7b889
Branch: refs/heads/llap
Commit: edb7b889336398f3c3b72b29eea85ef3457e6abb
Parents: ffce225
Author: Thejas Nair <th...@hortonworks.com>
Authored: Tue Jun 30 20:06:24 2015 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Tue Jun 30 20:06:24 2015 -0700
----------------------------------------------------------------------
.../hadoop/hive/metastore/ObjectStore.java | 26 +++++++-------------
1 file changed, 9 insertions(+), 17 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/edb7b889/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --git a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index 417ecc8..4273c0b 100644
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@ -23,7 +23,6 @@ import static org.apache.commons.lang.StringUtils.join;
import java.io.IOException;
import java.net.InetAddress;
import java.net.URI;
-import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -58,6 +57,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.ObjectPair;
import org.apache.hadoop.hive.common.classification.InterfaceAudience;
@@ -5568,10 +5568,8 @@ public class ObjectStore implements RawStore, Configurable {
URI locationURI = null;
String location = mDB.getLocationUri();
try {
- locationURI = new URI(location);
- } catch(URISyntaxException e) {
- badRecords.add(location);
- } catch (NullPointerException e) {
+ locationURI = new Path(location).toUri();
+ } catch (IllegalArgumentException e) {
badRecords.add(location);
}
if (locationURI == null) {
@@ -5631,10 +5629,8 @@ public class ObjectStore implements RawStore, Configurable {
if (parameters.containsKey(tblPropKey)) {
String tablePropLocation = parameters.get(tblPropKey);
try {
- tablePropLocationURI = new URI(tablePropLocation);
- } catch (URISyntaxException e) {
- badRecords.add(tablePropLocation);
- } catch (NullPointerException e) {
+ tablePropLocationURI = new Path(tablePropLocation).toUri();
+ } catch (IllegalArgumentException e) {
badRecords.add(tablePropLocation);
}
// if tablePropKey that was passed in lead to a valid URI resolution, update it if
@@ -5778,10 +5774,8 @@ public class ObjectStore implements RawStore, Configurable {
URI locationURI = null;
String location = mSDS.getLocation();
try {
- locationURI = new URI(location);
- } catch (URISyntaxException e) {
- badRecords.add(location);
- } catch (NullPointerException e) {
+ locationURI = new Path(location).toUri();
+ } catch (IllegalArgumentException e) {
badRecords.add(location);
}
if (locationURI == null) {
@@ -5859,10 +5853,8 @@ public class ObjectStore implements RawStore, Configurable {
String schemaLoc = mSerde.getParameters().get(serdeProp);
URI schemaLocURI = null;
try {
- schemaLocURI = new URI(schemaLoc);
- } catch (URISyntaxException e) {
- badRecords.add(schemaLoc);
- } catch (NullPointerException e) {
+ schemaLocURI = new Path(schemaLoc).toUri();
+ } catch (IllegalArgumentException e) {
badRecords.add(schemaLoc);
}
if (schemaLocURI == null) {
[03/25] hive git commit: HIVE-11095: SerDeUtils another bug ,
when Text is reused (Xiaowei via Xuefu)
Posted by se...@apache.org.
HIVE-11095: SerDeUtils another bug ,when Text is reused (Xiaowei via Xuefu)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9c64f937
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9c64f937
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9c64f937
Branch: refs/heads/llap
Commit: 9c64f9378143b6804c384f0333014b8f1cbd8edd
Parents: 97b4750
Author: Xuefu Zhang <xz...@Cloudera.com>
Authored: Tue Jun 30 05:23:32 2015 -0700
Committer: Xuefu Zhang <xz...@Cloudera.com>
Committed: Tue Jun 30 05:24:03 2015 -0700
----------------------------------------------------------------------
data/files/encoding-utf8.txt | 12 +++
.../insert_non_utf8_encoding_table.q | 20 +++++
.../insert_non_utf8_encoding_table.q.out | 89 ++++++++++++++++++++
.../apache/hadoop/hive/serde2/SerDeUtils.java | 2 +-
4 files changed, 122 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/9c64f937/data/files/encoding-utf8.txt
----------------------------------------------------------------------
diff --git a/data/files/encoding-utf8.txt b/data/files/encoding-utf8.txt
new file mode 100644
index 0000000..88bd256
--- /dev/null
+++ b/data/files/encoding-utf8.txt
@@ -0,0 +1,12 @@
+Tao,Li
+Wisgood
+Benguo,Me
+Xianqiang,Shen
+Wensheng,Wang
+Haijun,Qiao
+Shilong,Zhang
+Xiaoqing,You
+Aiqing,Song
+Zhenhua,Han
+Weiqi,Peng
+Hua,Li
http://git-wip-us.apache.org/repos/asf/hive/blob/9c64f937/ql/src/test/queries/clientpositive/insert_non_utf8_encoding_table.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/insert_non_utf8_encoding_table.q b/ql/src/test/queries/clientpositive/insert_non_utf8_encoding_table.q
new file mode 100644
index 0000000..0f9db02
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/insert_non_utf8_encoding_table.q
@@ -0,0 +1,20 @@
+drop table if exists table_with_utf8_encoding;
+
+create table table_with_utf8_encoding (name STRING)
+ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+ WITH SERDEPROPERTIES ('serialization.encoding'='utf-8');
+
+load data local inpath '../../data/files/encoding-utf8.txt' overwrite into table table_with_utf8_encoding;
+
+select * from table_with_utf8_encoding;
+
+drop table if exists table_with_non_utf8_encoding;
+
+create table table_with_non_utf8_encoding (name STRING)
+ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+ WITH SERDEPROPERTIES ('serialization.encoding'='ISO8859_1');
+
+insert overwrite table table_with_non_utf8_encoding select name from table_with_utf8_encoding;
+
+select * from table_with_non_utf8_encoding;
+
http://git-wip-us.apache.org/repos/asf/hive/blob/9c64f937/ql/src/test/results/clientpositive/insert_non_utf8_encoding_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/insert_non_utf8_encoding_table.q.out b/ql/src/test/results/clientpositive/insert_non_utf8_encoding_table.q.out
new file mode 100644
index 0000000..4d97d87
--- /dev/null
+++ b/ql/src/test/results/clientpositive/insert_non_utf8_encoding_table.q.out
@@ -0,0 +1,89 @@
+PREHOOK: query: drop table if exists table_with_utf8_encoding
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists table_with_utf8_encoding
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table table_with_utf8_encoding (name STRING)
+ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+ WITH SERDEPROPERTIES ('serialization.encoding'='utf-8')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_with_utf8_encoding
+POSTHOOK: query: create table table_with_utf8_encoding (name STRING)
+ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+ WITH SERDEPROPERTIES ('serialization.encoding'='utf-8')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_with_utf8_encoding
+PREHOOK: query: load data local inpath '../../data/files/encoding-utf8.txt' overwrite into table table_with_utf8_encoding
+PREHOOK: type: LOAD
+#### A masked pattern was here ####
+PREHOOK: Output: default@table_with_utf8_encoding
+POSTHOOK: query: load data local inpath '../../data/files/encoding-utf8.txt' overwrite into table table_with_utf8_encoding
+POSTHOOK: type: LOAD
+#### A masked pattern was here ####
+POSTHOOK: Output: default@table_with_utf8_encoding
+PREHOOK: query: select * from table_with_utf8_encoding
+PREHOOK: type: QUERY
+PREHOOK: Input: default@table_with_utf8_encoding
+#### A masked pattern was here ####
+POSTHOOK: query: select * from table_with_utf8_encoding
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@table_with_utf8_encoding
+#### A masked pattern was here ####
+Tao,Li
+Wisgood
+Benguo,Me
+Xianqiang,Shen
+Wensheng,Wang
+Haijun,Qiao
+Shilong,Zhang
+Xiaoqing,You
+Aiqing,Song
+Zhenhua,Han
+Weiqi,Peng
+Hua,Li
+PREHOOK: query: drop table if exists table_with_non_utf8_encoding
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists table_with_non_utf8_encoding
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create table table_with_non_utf8_encoding (name STRING)
+ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+ WITH SERDEPROPERTIES ('serialization.encoding'='ISO8859_1')
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@table_with_non_utf8_encoding
+POSTHOOK: query: create table table_with_non_utf8_encoding (name STRING)
+ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
+ WITH SERDEPROPERTIES ('serialization.encoding'='ISO8859_1')
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@table_with_non_utf8_encoding
+PREHOOK: query: insert overwrite table table_with_non_utf8_encoding select name from table_with_utf8_encoding
+PREHOOK: type: QUERY
+PREHOOK: Input: default@table_with_utf8_encoding
+PREHOOK: Output: default@table_with_non_utf8_encoding
+POSTHOOK: query: insert overwrite table table_with_non_utf8_encoding select name from table_with_utf8_encoding
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@table_with_utf8_encoding
+POSTHOOK: Output: default@table_with_non_utf8_encoding
+POSTHOOK: Lineage: table_with_non_utf8_encoding.name SIMPLE [(table_with_utf8_encoding)table_with_utf8_encoding.FieldSchema(name:name, type:string, comment:null), ]
+PREHOOK: query: select * from table_with_non_utf8_encoding
+PREHOOK: type: QUERY
+PREHOOK: Input: default@table_with_non_utf8_encoding
+#### A masked pattern was here ####
+POSTHOOK: query: select * from table_with_non_utf8_encoding
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@table_with_non_utf8_encoding
+#### A masked pattern was here ####
+Tao,Li
+Wisgood
+Benguo,Me
+Xianqiang,Shen
+Wensheng,Wang
+Haijun,Qiao
+Shilong,Zhang
+Xiaoqing,You
+Aiqing,Song
+Zhenhua,Han
+Weiqi,Peng
+Hua,Li
http://git-wip-us.apache.org/repos/asf/hive/blob/9c64f937/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
index 40ede1a..c65174e 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/SerDeUtils.java
@@ -555,6 +555,6 @@ public final class SerDeUtils {
}
public static Text transformTextFromUTF8(Text text, Charset targetCharset) {
- return new Text(new String(text.getBytes()).getBytes(targetCharset));
+ return new Text(new String(text.getBytes(), 0, text.getLength()).getBytes(targetCharset));
}
}