From 349b1c23eb9653a7ef0206e9c2a94178953e134f Mon Sep 17 00:00:00 2001 From: Hyunsik Choi Date: Thu, 12 Mar 2015 02:53:45 -0700 Subject: [PATCH 01/55] Initial prototype. --- .../java/org/apache/tajo/conf/TajoConf.java | 2 + tajo-core/pom.xml | 6 + .../function/python/JythonScriptEngine.java | 431 ++++++++++++++++++ .../function/python/TajoScriptEngine.java | 94 ++++ .../tajo/engine/query/QueryContext.java | 6 + .../python/TestJythonScriptEngine.java | 39 ++ .../src/test/resources/python/__init__.py | 0 .../src/test/resources/python/pig_util.py | 85 ++++ tajo-core/src/test/resources/python/test1.py | 5 + 9 files changed, 668 insertions(+) create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java create mode 100644 tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java create mode 100644 tajo-core/src/test/resources/python/__init__.py create mode 100644 tajo-core/src/test/resources/python/pig_util.py create mode 100644 tajo-core/src/test/resources/python/test1.py diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 4ed8097b4f..122febec09 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -287,6 +287,8 @@ public static enum ConfVars implements ConfigKey { // Geo IP GEOIP_DATA("tajo.function.geoip-database-location", ""), + PYTHON_CMD_ARGS_REMAINDERS("tajo.function.python.args-reminders", ""), + ///////////////////////////////////////////////////////////////////////////////// // User Session Configuration // diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index ce9db73ad6..7de9bffb0c 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -450,6 +450,12 @@ jcip-annotations test + + + org.python + jython-standalone + 2.7-b3 + diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java new file mode 100644 index 0000000000..88d6703c35 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java @@ -0,0 +1,431 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.python; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.engine.query.QueryContext; +import org.python.core.ClasspathPyImporter; +import org.python.core.Py; +import org.python.core.PyException; +import org.python.core.PyFrame; +import org.python.core.PyFunction; +import org.python.core.PyInteger; +import org.python.core.PyJavaPackage; +import org.python.core.PyObject; +import org.python.core.PyString; +import org.python.core.PyStringMap; +import org.python.core.PySystemState; +import org.python.core.PyTuple; +import org.python.modules.zipimport.zipimporter; +import org.python.util.PythonInterpreter; + +import javax.script.ScriptEngine; + +/** + * Implementation of the script engine for Jython + */ +public class JythonScriptEngine extends TajoScriptEngine { + private static final Log LOG = LogFactory.getLog(JythonScriptEngine.class); + + /** + * Language Interpreter Uses static holder pattern + */ + private static class Interpreter { + static final PythonInterpreter interpreter; + static final ArrayList filesLoaded = new ArrayList(); + static final String JVM_JAR; + + static { + // should look like: file:JVM_JAR!/java/lang/Object.class + String rpath = Object.class.getResource("Object.class").getPath(); + JVM_JAR = rpath.replaceAll("^file:(.*)!/java/lang/Object.class$", "$1"); + + // Determine if a usable python.cachedir has been provided + // if not, certain uses of jython's import will not work e.g., so create a tmp dir + // - from some.package import * + // - import non.jvm.package + try { + String skip = System.getProperty(PySystemState.PYTHON_CACHEDIR_SKIP, "false"); + if (skip.equalsIgnoreCase("true")) { + LOG.warn("jython cachedir skipped, jython may not work"); + } else { + File tmp = null; + String cdir = System.getProperty(PySystemState.PYTHON_CACHEDIR); + if (cdir != null) { + tmp = new File(cdir); + if (tmp.canWrite() == false) { + LOG.error("CACHEDIR: not writable"); + throw new RuntimeException("python.cachedir not writable: " + cdir); + } + } + if (tmp == null) { + tmp = File.createTempFile("pig_jython_", ""); + tmp.delete(); + if (tmp.mkdirs() == false) { + LOG.warn("unable to create a tmp dir for the cache, jython may not work"); + } else { + LOG.info("created tmp python.cachedir=" + tmp); + System.setProperty(PySystemState.PYTHON_CACHEDIR, tmp.getAbsolutePath()); + } + Runtime.getRuntime().addShutdownHook(new DirDeleter(tmp)); + } + } + // local file system import path elements: current dir, JYTHON_HOME/Lib + Py.getSystemState().path.append(new PyString(System.getProperty("user.dir"))); + String jyhome = System.getenv("JYTHON_HOME"); + if (jyhome != null) { + Py.getSystemState().path.append(new PyString(jyhome + File.separator + "Lib")); + } + LOG.info(ClassLoader.getSystemResource("python")); + Py.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").toString())); + } catch (Exception e) { + LOG.warn("issue with jython cache dir", e); + } + + // cacdedir now configured, allocate the python interpreter + interpreter = new PythonInterpreter(); + } + + /** + * ensure the decorator functions are defined in the interpreter, and + * manage the module import dependencies. + * @param path location of a file to exec in the interpreter + * @param queryContext if non-null, module import state is tracked + * @throws IOException + */ + static synchronized void init(String path, QueryContext queryContext) throws IOException { + // Decorators - + // "schemaFunction" + // "outputSchema" + // "outputSchemaFunction" + + if (!filesLoaded.contains(path)) { + // attempt addition of schema decorator handler, fail silently + interpreter.exec("def outputSchema(schema_def):\n" + + " def decorator(func):\n" + + " func.outputSchema = schema_def\n" + + " return func\n" + + " return decorator\n\n"); + + interpreter.exec("def outputSchemaFunction(schema_def):\n" + + " def decorator(func):\n" + + " func.outputSchemaFunction = schema_def\n" + + " return func\n" + + " return decorator\n"); + + interpreter.exec("def schemaFunction(schema_def):\n" + + " def decorator(func):\n" + + " func.schemaFunction = schema_def\n" + + " return func\n" + + " return decorator\n\n"); + + InputStream is = getScriptAsStream(path); + if (is == null) { + throw new IllegalStateException("unable to create a stream for path: " + path); + } + try { + execfile(is, path, queryContext); + } finally { + is.close(); + } + } + } + + /** + * does not call script.close() + * @param script + * @param path + * @param queryContext + * @throws Exception + */ + static void execfile(InputStream script, String path, QueryContext queryContext) throws RuntimeException { + try { + + if( queryContext != null ) { + String [] argv; + try { +// argv = (String[])ObjectSerializer.deserialize( +// queryContext.get(ConfVars.PYTHON_CMD_ARGS_REMAINDERS)); + argv = new String [] {}; + } catch (Exception e) { + throw new RuntimeException("Cannot deserialize command line arguments", e); + } + PySystemState state = Py.getSystemState(); + state.argv.clear(); + if( argv != null ) { + for (String str : argv ) { + state.argv.append(new PyString(str)); + } + } else { + LOG.warn(ConfVars.PYTHON_CMD_ARGS_REMAINDERS.name() + + " is empty. This is not expected unless on testing." ); + } + } + + // determine the current module state + Map before = queryContext != null ? getModuleState() : null; + if (before != null) { + // os.py, stax.py and posixpath.py are part of the initial state + // if Lib directory is present and without including os.py, modules + // which import os fail + Set includePyModules = new HashSet(); + for (String key : before.keySet()) { + // $py.class is created if Lib folder is writable + if (key.endsWith(".py") || key.endsWith("$py.class")) { + includePyModules.add(key); + } + } + before.keySet().removeAll(includePyModules); + } + + LOG.info(ClassLoader.getSystemResource("python").getPath()); + interpreter.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").getPath())); + + // exec the code, arbitrary imports are processed + interpreter.execfile(script, path); + + // determine the 'post import' module state + Map after = queryContext != null ? getModuleState() : null; + + // add the module files to the context + if (after != null && queryContext != null) { + after.keySet().removeAll(before.keySet()); + for (Map.Entry entry : after.entrySet()) { + String modulename = entry.getKey(); + String modulepath = entry.getValue(); + if (modulepath.equals(JVM_JAR)) { + continue; + } else if (modulepath.endsWith(".jar") || modulepath.endsWith(".zip")) { + queryContext.addScriptJar(modulepath); + } else { + queryContext.addScriptFile(modulename, modulepath); + } + } + } + } catch (PyException e) { + if (e.match(Py.SystemExit)) { + PyObject value = e.value; + if (PyException.isExceptionInstance(e.value)) { + value = value.__findattr__("code"); + } + if (new PyInteger(0).equals(value)) { + LOG.info("Script invoked sys.exit(0)"); + return; + } + } + String message = "Python Error. " + e; + //throw new ExecException(message, 1121, e); + throw new RuntimeException(message, e); + } + } + + static String get(String name) { + return interpreter.get(name).toString(); + } + + static void setMain(boolean isMain) { + if (isMain) { + interpreter.set("__name__", "__main__"); + } else { + interpreter.set("__name__", "__lib__"); + } + } + + /** + * get the state of modules currently loaded + * @return a map of module name to module file (absolute path) + */ + private static Map getModuleState() { + // determine the current module state + Map files = new HashMap(); + PyStringMap modules = (PyStringMap) Py.getSystemState().modules; + for (PyObject kvp : modules.iteritems().asIterable()) { + PyTuple tuple = (PyTuple) kvp; + String name = tuple.get(0).toString(); + Object value = tuple.get(1); + // inspect the module to determine file location and status + try { + Object fileEntry = null; + Object loader = null; + if (value instanceof PyJavaPackage ) { + fileEntry = ((PyJavaPackage) value).__file__; + } else if (value instanceof PyObject) { + // resolved through the filesystem (or built-in) + PyObject dict = ((PyObject) value).getDict(); + if (dict != null) { + fileEntry = dict.__finditem__("__file__"); + loader = dict.__finditem__("__loader__"); + } // else built-in + } // else some system module? + + if (fileEntry != null) { + File file = resolvePyModulePath(fileEntry.toString(), loader); + if (file.exists()) { + String apath = file.getAbsolutePath(); + if (apath.endsWith(".jar") || apath.endsWith(".zip")) { + // jar files are simple added to the pigContext + files.put(apath, apath); + } else { + // determine the relative path that the file should have in the jar + int pos = apath.lastIndexOf(File.separatorChar + name.replace('.', File.separatorChar)); + if (pos > 0) { + files.put(apath.substring(pos + 1), apath); + } else { + files.put(apath, apath); + } + } + } else { + LOG.warn("module file does not exist: " + name + ", " + file); + } + } // else built-in + } catch (Exception e) { + LOG.warn("exception while retrieving module state: " + value, e); + } + } + return files; + } + } + + private static File resolvePyModulePath(String path, Object loader) { + File file = new File(path); + if (!file.exists() && loader != null) { + if(path.startsWith(ClasspathPyImporter.PYCLASSPATH_PREFIX) && loader instanceof ClasspathPyImporter) { + path = path.replaceFirst(ClasspathPyImporter.PYCLASSPATH_PREFIX, ""); + URL resource = ScriptEngine.class.getResource(path); + if (resource == null) { + resource = ScriptEngine.class.getResource(File.separator + path); + } + if (resource != null) { + return new File(resource.getFile()); + } + } else if (loader instanceof zipimporter) { + zipimporter importer = (zipimporter) loader; + return new File(importer.archive); + } //JavaImporter?? + } + return file; + } + + /** + * Gets the Python function object. + * @param path Path of the jython script file containing the function. + * @param functionName Name of the function + * @return a function object + * @throws IOException + */ + public static PyFunction getFunction(String path, String functionName) throws IOException { + Interpreter.setMain(false); + Interpreter.init(path, null); + return (PyFunction) Interpreter.interpreter.get(functionName); + } + + @Override + protected String getScriptingLang() { + return "jython"; + } + + @Override + protected Map getParamsFromVariables() throws IOException { + PyFrame frame = Py.getFrame(); + @SuppressWarnings("unchecked") + List locals = ((PyStringMap) frame.getLocals()).items(); + Map vars = new HashMap(); + for (PyTuple item : locals) { + String key = (String) item.get(0); + Object obj = item.get(1); + if (obj != null) { + String value = item.get(1).toString(); + vars.put(key, value); + } + } + return vars; + } + + private static final Pattern p = Pattern.compile("^\\s*def\\s+(\\w+)\\s*.+"); + private static final Pattern p1 = Pattern.compile("^\\s*if\\s+__name__\\s+==\\s+[\"']__main__[\"']\\s*:\\s*$"); + + private static boolean hasFunction(InputStream is) throws IOException { + boolean hasFunction = false; + boolean hasMain = false; + InputStreamReader in = new InputStreamReader(is); + BufferedReader br = new BufferedReader(in); + String line = br.readLine(); + while (line != null) { + if (p.matcher(line).matches()) { + hasFunction = true; + } else if (p1.matcher(line).matches()) { + hasMain = true; + } + line = br.readLine(); + } + if (hasFunction && !hasMain) { + String msg = "Embedded script cannot mix UDFs with top level code. " + + "Please use if __name__ == '__main__': construct"; + throw new IOException(msg); + } + return hasFunction; + } + + /** + * File.deleteOnExit(File) does not work for a non-empty directory. This + * Thread is used to clean up the python.cachedir (if it was a tmp dir + * created by the Engine) + */ + private static class DirDeleter extends Thread { + private final File dir; + public DirDeleter(final File file) { + dir = file; + } + @Override + public void run() { + try { + delete(dir); + } catch (Exception e) { + LOG.warn("on cleanup", e); + } + } + private static boolean delete(final File file) { + if (file.isDirectory()) { + for (File f : file.listFiles()) { + delete(f); + } + } + return file.delete(); + } + } +} + diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java new file mode 100644 index 0000000000..1d69e84817 --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.python; + +import org.apache.hadoop.util.Shell; + +import javax.script.ScriptEngine; +import java.io.*; +import java.util.Map; + +public abstract class TajoScriptEngine { + /** + * open a stream load a script locally or in the classpath + * @param scriptPath the path of the script + * @return a stream (it is the responsibility of the caller to close it) + * @throws IllegalStateException if we could not open a stream + */ + public static InputStream getScriptAsStream(String scriptPath) { + //protected static InputStream getScriptAsStream(String scriptPath) { + InputStream is = null; + File file = new File(scriptPath); + if (file.exists()) { + try { + is = new FileInputStream(file); + } catch (FileNotFoundException e) { + throw new IllegalStateException("could not find existing file "+scriptPath, e); + } + } else { + if (Shell.WINDOWS && scriptPath.charAt(1)==':') { + scriptPath = scriptPath.charAt(0) + scriptPath.substring(2); + } + // Try system, current and context classloader. + is = ScriptEngine.class.getResourceAsStream(scriptPath); + if (is == null) { + is = getResourceUsingClassLoader(scriptPath, ScriptEngine.class.getClassLoader()); + } + if (is == null) { + is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); + } + if (is == null && !file.isAbsolute()) { + String path = "/" + scriptPath; + is = ScriptEngine.class.getResourceAsStream(path); + if (is == null) { + is = getResourceUsingClassLoader(path, ScriptEngine.class.getClassLoader()); + } + if (is == null) { + is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); + } + } + } + + // TODO: discuss if we want to add logic here to load a script from HDFS + + if (is == null) { + throw new IllegalStateException( + "Could not initialize interpreter (from file system or classpath) with " + scriptPath); + } + return is; + } + + private static InputStream getResourceUsingClassLoader(String fullFilename, ClassLoader loader) { + if (loader != null) { + return loader.getResourceAsStream(fullFilename); + } + return null; + } + + /** + * Gets ScriptEngine classname or keyword for the scripting language + */ + protected abstract String getScriptingLang(); + + /** + * Returns a map from local variable names to their values + * @throws java.io.IOException + */ + protected abstract Map getParamsFromVariables() throws IOException; +} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java index 7b3c00db30..a013dc0ac8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java @@ -166,4 +166,10 @@ public void setInsert() { public boolean isInsert() { return isCommandType(NodeType.INSERT.name()); } + + public void addScriptJar(String modulepath) { + } + + public void addScriptFile(String modulename, String modulepath) { + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java new file mode 100644 index 0000000000..820ed249ce --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.python; + +import junit.framework.TestCase; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.util.FileUtil; +import org.python.core.PyFunction; + +import java.net.URL; + +public class TestJythonScriptEngine extends TestCase { + static final Log LOG = LogFactory.getLog(TestJythonScriptEngine.class); + + public void testGetFunction() throws Exception { + URL url = FileUtil.getResourcePath("python/test1.py"); + LOG.info("File path: " + url); + PyFunction function = JythonScriptEngine.getFunction(url.getPath(), "return_one"); + LOG.info(function.getType()); + function.__call__(); + } +} \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/__init__.py b/tajo-core/src/test/resources/python/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/tajo-core/src/test/resources/python/pig_util.py b/tajo-core/src/test/resources/python/pig_util.py new file mode 100644 index 0000000000..fbbfef846d --- /dev/null +++ b/tajo-core/src/test/resources/python/pig_util.py @@ -0,0 +1,85 @@ +import logging + +class udf_logging(object): + udf_log_level = logging.INFO + + @classmethod + def set_log_level_error(cls): + cls.udf_log_level = logging.ERROR + + @classmethod + def set_log_level_warn(cls): + cls.udf_log_level = logging.WARN + + @classmethod + def set_log_level_info(cls): + cls.udf_log_level = logging.INFO + + @classmethod + def set_log_level_debug(cls): + cls.udf_log_level = logging.DEBUG + +def outputSchema(schema_str): + def wrap(f): + def wrapped_f(*args): + return f(*args) + return wrapped_f + return wrap + +def write_user_exception(filename, stream_err_output, num_lines_offset_trace=0): + import sys + import traceback + import inspect + (t, v, tb) = sys.exc_info() + name = t.__name__ + record_error = False + + if name in ['SyntaxError', 'IndentationError']: + syntax_error_values = v.args + user_line_number = syntax_error_values[1][1] - num_lines_offset_trace + error_message = "%s: %s\n\tFile: %s, line %s column %s\n\t%s" % \ + (name, + syntax_error_values[0], + syntax_error_values[1][0], + user_line_number, + syntax_error_values[1][2], + syntax_error_values[1][3]) + else: + error_message = "%s: %s\n" % (name, v) + user_line_number = None + while 1: + e_file_name = tb.tb_frame.f_code.co_filename + if e_file_name.find(filename) > 0: + record_error = True + if not record_error: + if not tb.tb_next: + break + tb = tb.tb_next + continue + + line_number = tb.tb_lineno + mod = inspect.getmodule(tb) + if mod: + lines, offset = inspect.getsourcelines(mod) + line = lines[line_number - offset - 1] + else: + #Useful to catch exceptions with an invalid module (like syntax + #errors) + lines, offset = inspect.getsourcelines(tb.tb_frame) + if (line_number - 1) >= len(lines): + line = "Unknown Line" + else: + line = lines[line_number - 1] + + user_line_number = line_number - num_lines_offset_trace + func_name = tb.tb_frame.f_code.co_name + error_message += 'File %s, line %s, in %s\n\t%s\n' % \ + (e_file_name, user_line_number, func_name, line) + if not tb.tb_next: + break + tb = tb.tb_next + if name in ['UnicodeEncodeError']: + error_message += "\nTo print a unicode string in your udf use encode('utf-8'). Example: \n\tprint 'Example'.encode('utf-8')" + if user_line_number: + stream_err_output.write("%s\n" % user_line_number) + stream_err_output.write("%s\n" % error_message) diff --git a/tajo-core/src/test/resources/python/test1.py b/tajo-core/src/test/resources/python/test1.py new file mode 100644 index 0000000000..d62a100d78 --- /dev/null +++ b/tajo-core/src/test/resources/python/test1.py @@ -0,0 +1,5 @@ +from pig_util import outputSchema + +@outputSchema('value:int') +def return_one(): + return 1 From 104fa93d4e4fb8752679c94ea06c0357692a95d5 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 23 Mar 2015 22:59:15 +0900 Subject: [PATCH 02/55] TAJO-1344 --- .../src/test/resources/python/__init__.py | 17 ++++++++++++++++ .../python/{pig_util.py => tajo_util.py} | 18 +++++++++++++++++ tajo-core/src/test/resources/python/test1.py | 20 ++++++++++++++++++- 3 files changed, 54 insertions(+), 1 deletion(-) rename tajo-core/src/test/resources/python/{pig_util.py => tajo_util.py} (79%) diff --git a/tajo-core/src/test/resources/python/__init__.py b/tajo-core/src/test/resources/python/__init__.py index e69de29bb2..67bb99de4d 100644 --- a/tajo-core/src/test/resources/python/__init__.py +++ b/tajo-core/src/test/resources/python/__init__.py @@ -0,0 +1,17 @@ +""" +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. +""" \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/pig_util.py b/tajo-core/src/test/resources/python/tajo_util.py similarity index 79% rename from tajo-core/src/test/resources/python/pig_util.py rename to tajo-core/src/test/resources/python/tajo_util.py index fbbfef846d..a505145a52 100644 --- a/tajo-core/src/test/resources/python/pig_util.py +++ b/tajo-core/src/test/resources/python/tajo_util.py @@ -1,3 +1,21 @@ +""" +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you under the Apache License, Version 2.0 (the +"License"); you may not use this file except in compliance +with the License. You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +""" + import logging class udf_logging(object): diff --git a/tajo-core/src/test/resources/python/test1.py b/tajo-core/src/test/resources/python/test1.py index d62a100d78..47abcddd1f 100644 --- a/tajo-core/src/test/resources/python/test1.py +++ b/tajo-core/src/test/resources/python/test1.py @@ -1,4 +1,22 @@ -from pig_util import outputSchema +""" +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. +""" + +from tajo_util import outputSchema @outputSchema('value:int') def return_one(): From 9241d1ef5a95479772cf192f942905347c3729a4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 24 Mar 2015 17:05:31 +0900 Subject: [PATCH 03/55] TAJO-1344 --- .../plan/expr/GeneralPythonFunctionEval.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java new file mode 100644 index 0000000000..a1a895900c --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.expr; + +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.storage.Tuple; + +public class GeneralPythonFunctionEval extends FunctionEval { + + public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { + super(EvalType.FUNCTION, funcDesc, argEvals); + } + + @Override + public Datum eval(Schema schema, Tuple tuple) { + return null; + } +} From b3d09b82c0288ef6e9c6e85715c50bf5578eea0b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 25 Mar 2015 19:03:48 +0900 Subject: [PATCH 04/55] TAJO-1344 --- .../tajo/function/FunctionInvocation.java | 20 +++- .../tajo/function/PythonInvocationDesc.java | 37 +++++++ .../java/org/apache/tajo/conf/TajoConf.java | 2 + .../tajo/engine/function/FunctionLoader.java | 43 +++++++- .../function/python/JythonScriptEngine.java | 104 +++++++++++++----- .../engine/function/python/JythonUtils.java | 47 ++++++++ .../python/OptionalFunctionContext.java | 49 +++++++++ .../function/python/TajoScriptEngine.java | 32 ++++++ .../tajo/engine/query/QueryContext.java | 6 - .../org/apache/tajo/master/TajoMaster.java | 25 ++++- .../org/apache/tajo/worker/TajoWorker.java | 32 +++++- .../org/apache/tajo/TajoTestingCluster.java | 2 + .../apache/tajo/engine/eval/ExprTestBase.java | 3 +- .../python/TestJythonScriptEngine.java | 7 +- .../src/test/resources/python/__init__.py | 32 +++--- .../src/test/resources/python/tajo_util.py | 32 +++--- tajo-core/src/test/resources/python/test1.py | 50 +++++---- .../plan/expr/GeneralPythonFunctionEval.java | 1 + 18 files changed, 427 insertions(+), 97 deletions(-) create mode 100644 tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonUtils.java create mode 100644 tajo-core/src/main/java/org/apache/tajo/engine/function/python/OptionalFunctionContext.java diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java index 653bdb6906..3d92a98445 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java @@ -35,6 +35,8 @@ public class FunctionInvocation implements ProtoObject StaticMethodInvocationDesc scalarJIT; @Expose ClassBaseInvocationDesc aggregationJIT; + @Expose + PythonInvocationDesc python; public FunctionInvocation() { } @@ -55,6 +57,7 @@ public FunctionInvocation(FunctionInvocationProto proto) { if (proto.hasAggregationJIT()) { this.aggregationJIT = new ClassBaseInvocationDesc(proto.getAggregation()); } + // TODO: add PythonInvocationDesc to FucntionInvocationProto } public boolean isAvailable() { @@ -121,6 +124,18 @@ public ClassBaseInvocationDesc getAggregationJIT() { return aggregationJIT; } + public boolean hasPython() { + return python != null; + } + + public void setPython(PythonInvocationDesc python) { + this.python = python; + } + + public PythonInvocationDesc getPython() { + return python; + } + @Override public FunctionInvocationProto getProto() { FunctionInvocationProto.Builder builder = FunctionInvocationProto.newBuilder(); @@ -139,16 +154,17 @@ public FunctionInvocationProto getProto() { if (hasAggregationJIT()) { builder.setAggregationJIT(aggregationJIT.getProto()); } + // TODO: add PythonInvocationDesc to FucntionInvocationProto return builder.build(); } @Override public int hashCode() { - return Objects.hashCode(legacy, scalar, scalarJIT); + return Objects.hashCode(legacy, scalar, scalarJIT, python); } public String toString() { return "legacy=" + hasLegacy() + ",scalar=" + hasScalar() + ",agg=" + hasAggregation() + - ",scalarJIT=" + hasScalarJIT() + ",aggJIT=" + hasAggregationJIT(); + ",scalarJIT=" + hasScalarJIT() + ",aggJIT=" + hasAggregationJIT() + ",python=" + hasPython(); } } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java new file mode 100644 index 0000000000..9df81e1f5e --- /dev/null +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.function; + +public class PythonInvocationDesc { + private final String funcName; + private final String filePath; + + public PythonInvocationDesc(String funcName, String filePath) { + this.funcName = funcName; + this.filePath = filePath; + } + + public String getName() { + return funcName; + } + + public String getPath() { + return filePath; + } +} diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 5f29dc46f5..cb4dcb050b 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -289,7 +289,9 @@ public static enum ConfVars implements ConfigKey { // Geo IP GEOIP_DATA("tajo.function.geoip-database-location", ""), + // Python UDF PYTHON_CMD_ARGS_REMAINDERS("tajo.function.python.args-reminders", ""), + PYTHON_CODE_DIR("tajo.function.python.code-dir", ""), ///////////////////////////////////////////////////////////////////////////////// // User Session Configuration diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index 3b3e7c7237..f8f6a778f1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -25,16 +25,24 @@ import org.apache.commons.collections.Predicate; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.function.annotation.Description; import org.apache.tajo.engine.function.annotation.ParamOptionTypes; import org.apache.tajo.engine.function.annotation.ParamTypes; +import org.apache.tajo.engine.function.python.JythonScriptEngine; +import org.apache.tajo.engine.function.python.OptionalFunctionContext; import org.apache.tajo.function.*; import org.apache.tajo.util.ClassUtil; +import org.apache.tajo.util.TUtil; +import java.io.IOException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.util.*; @@ -44,8 +52,9 @@ public class FunctionLoader { private static Log LOG = LogFactory.getLog(FunctionLoader.class); + public static final String PYTHON_FUNCTION_NAMESPACE = "python"; - public static Collection load() { + public static Map load() { Map map = Maps.newHashMap(); List dd = Lists.newArrayList(); @@ -66,7 +75,37 @@ public static Collection load() { } } - return map.values(); + return map; + } + + public static Map loadOptionalFunctions(TajoConf conf, + OptionalFunctionContext context, + Map functionMap) + throws IOException { + + String[] codePaths = conf.getStrings(TajoConf.ConfVars.PYTHON_CODE_DIR.varname); + if (codePaths != null) { + FileSystem localFS = FileSystem.getLocal(conf); + for (String codePathStr : codePaths) { + Path codePath = new Path(codePathStr); + List filePaths = TUtil.newList(); + if (localFS.isDirectory(codePath)) { + for (FileStatus file : localFS.listStatus(codePath)) { + filePaths.add(file.getPath()); + } + } else { + filePaths.add(codePath); + } + for (Path filePath : filePaths) { + for (FunctionDesc f : JythonScriptEngine.registerFunctions(context, filePath.toString(), + FunctionLoader.PYTHON_FUNCTION_NAMESPACE)) { + functionMap.put(f.getSignature(), f); + LOG.info(f); + } + } + } + } + return functionMap; } public static Set findScalarFunctions() { diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java index 88d6703c35..0731f89767 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java @@ -20,7 +20,6 @@ import java.io.BufferedReader; import java.io.File; -import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; @@ -35,21 +34,16 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.conf.TajoConf.ConfVars; -import org.apache.tajo.engine.query.QueryContext; -import org.python.core.ClasspathPyImporter; -import org.python.core.Py; -import org.python.core.PyException; -import org.python.core.PyFrame; -import org.python.core.PyFunction; -import org.python.core.PyInteger; -import org.python.core.PyJavaPackage; -import org.python.core.PyObject; -import org.python.core.PyString; -import org.python.core.PyStringMap; -import org.python.core.PySystemState; -import org.python.core.PyTuple; +import org.apache.tajo.function.FunctionInvocation; +import org.apache.tajo.function.FunctionSignature; +import org.apache.tajo.function.FunctionSupplement; +import org.apache.tajo.function.PythonInvocationDesc; +import org.apache.tajo.util.TUtil; +import org.python.core.*; import org.python.modules.zipimport.zipimporter; import org.python.util.PythonInterpreter; @@ -60,6 +54,7 @@ */ public class JythonScriptEngine extends TajoScriptEngine { private static final Log LOG = LogFactory.getLog(JythonScriptEngine.class); + public static final String NAMESPACE_SEPARATOR = "."; /** * Language Interpreter Uses static holder pattern @@ -127,7 +122,7 @@ private static class Interpreter { * @param queryContext if non-null, module import state is tracked * @throws IOException */ - static synchronized void init(String path, QueryContext queryContext) throws IOException { + static synchronized void init(OptionalFunctionContext context, String path) throws IOException { // Decorators - // "schemaFunction" // "outputSchema" @@ -158,7 +153,7 @@ static synchronized void init(String path, QueryContext queryContext) throws IOE throw new IllegalStateException("unable to create a stream for path: " + path); } try { - execfile(is, path, queryContext); + execfile(context, is, path); } finally { is.close(); } @@ -172,10 +167,10 @@ static synchronized void init(String path, QueryContext queryContext) throws IOE * @param queryContext * @throws Exception */ - static void execfile(InputStream script, String path, QueryContext queryContext) throws RuntimeException { + static void execfile(OptionalFunctionContext context, InputStream script, String path) throws RuntimeException { try { - if( queryContext != null ) { + if( context != null ) { String [] argv; try { // argv = (String[])ObjectSerializer.deserialize( @@ -197,7 +192,7 @@ static void execfile(InputStream script, String path, QueryContext queryContext) } // determine the current module state - Map before = queryContext != null ? getModuleState() : null; + Map before = context != null ? getModuleState() : null; if (before != null) { // os.py, stax.py and posixpath.py are part of the initial state // if Lib directory is present and without including os.py, modules @@ -219,10 +214,10 @@ static void execfile(InputStream script, String path, QueryContext queryContext) interpreter.execfile(script, path); // determine the 'post import' module state - Map after = queryContext != null ? getModuleState() : null; + Map after = context != null ? getModuleState() : null; // add the module files to the context - if (after != null && queryContext != null) { + if (after != null && context != null) { after.keySet().removeAll(before.keySet()); for (Map.Entry entry : after.entrySet()) { String modulename = entry.getKey(); @@ -230,9 +225,10 @@ static void execfile(InputStream script, String path, QueryContext queryContext) if (modulepath.equals(JVM_JAR)) { continue; } else if (modulepath.endsWith(".jar") || modulepath.endsWith(".zip")) { - queryContext.addScriptJar(modulepath); +// context.addScriptJar(modulepath); + throw new RuntimeException("jar and zip script files are not supported"); } else { - queryContext.addScriptFile(modulename, modulepath); + context.addScriptFile(modulename, modulepath); } } } @@ -349,7 +345,7 @@ private static File resolvePyModulePath(String path, Object loader) { */ public static PyFunction getFunction(String path, String functionName) throws IOException { Interpreter.setMain(false); - Interpreter.init(path, null); + Interpreter.init(null, path); return (PyFunction) Interpreter.interpreter.get(functionName); } @@ -427,5 +423,63 @@ private static boolean delete(final File file) { return file.delete(); } } + + // @Override +// public void registerFunctions(String path, String namespace, QueryContext context) + public static Set registerFunctions(OptionalFunctionContext context, String path, String namespace) + throws IOException { + Interpreter.setMain(false); + Interpreter.init(context, path); +// context.addScriptJar(getJarPath(PythonInterpreter.class)); + PythonInterpreter pi = Interpreter.interpreter; + @SuppressWarnings("unchecked") + List locals = ((PyStringMap) pi.getLocals()).items(); + namespace = (namespace == null) ? "" : namespace + NAMESPACE_SEPARATOR; + Set functionDescs = TUtil.newHashSet(); + + for (PyTuple item : locals) { + String key = (String) item.get(0); + Object value = item.get(1); + if (!key.startsWith("__") && !key.equals("schemaFunction") + && !key.equals("outputSchema") + && !key.equals("outputSchemaFunction") + && (value instanceof PyFunction) + && (((PyFunction)value).__findattr__("schemaFunction".intern())== null)) { + PyFunction pyFunction = (PyFunction) value; + PyObject obj = pyFunction.__findattr__("outputSchema".intern()); + TajoDataTypes.Type returnType; + if(obj != null) { +// Utils.getSchemaFromString(obj.toString()); + LOG.info("outputSchema: " + obj.toString()); + String[] types = obj.toString().split(","); + if (types.length > 1) { + throw new IOException("Multiple return type is not supported"); + } + returnType = TajoDataTypes.Type.valueOf(types[0].trim().toUpperCase()); + } else { + // the default return type is the byte array + returnType = TajoDataTypes.Type.BLOB; + } + int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; + LOG.info("co_argcount: " + paramNum); + TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; + for (int i = 0; i < paramNum; i++) { + paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); + } + + FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, key, + TajoDataTypes.DataType.newBuilder().setType(returnType).build(), paramTypes); + FunctionInvocation invocation = new FunctionInvocation(); + invocation.setPython(new PythonInvocationDesc(key, path)); + FunctionSupplement supplement = new FunctionSupplement(); + functionDescs.add(new FunctionDesc(signature, invocation, supplement)); + LOG.info("Register scripting UDF: " + namespace + key); + } + } + + context.addScriptFile(path); + Interpreter.setMain(true); + return functionDescs; + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonUtils.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonUtils.java new file mode 100644 index 0000000000..05ef6bbbae --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonUtils.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.python; + +import org.apache.tajo.catalog.Schema; + +import java.util.HashMap; + +public class JythonUtils { + +// /** +// * @param schemaString a String representation of the Schema without +// * any enclosing curly-braces.Not for use with +// * Schema#toString +// * @return Schema instance +// * @throws ParserException +// */ +// public static Schema getSchemaFromString(String schemaString) { +// LogicalSchema schema = parseSchema(schemaString); +// Schema result = org.apache.pig.newplan.logical.Util.translateSchema(schema); +// Schema.setSchemaDefaultType(result, DataType.BYTEARRAY); +// return result; +// } +// +// public static LogicalSchema parseSchema(String schemaString) { +// QueryParserDriver queryParser = new QueryParserDriver( new PigContext(), +// "util", new HashMap() ) ; +// LogicalSchema schema = queryParser.parseSchema(schemaString); +// return schema; +// } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/OptionalFunctionContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/OptionalFunctionContext.java new file mode 100644 index 0000000000..268149f90a --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/OptionalFunctionContext.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function.python; + +import java.io.File; +import java.util.LinkedHashMap; +import java.util.Map; + +public class OptionalFunctionContext { + + private Map aliasedScriptFiles = new LinkedHashMap(); + + /** + * this method adds script files that must be added to the shipped jar + * named differently from their local fs path. + * @param name name in the jar + * @param path path on the local fs + */ + public void addScriptFile(String name, String path) { + if (path != null) { + aliasedScriptFiles.put(name.replaceFirst("^/", "").replaceAll(":", ""), new File(path)); + } + } + + /** + * calls: addScriptFile(path, new File(path)), ensuring that a given path is + * added to the jar at most once. + * @param path + */ + public void addScriptFile(String path) { + addScriptFile(path, path); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java index 1d69e84817..ff382139f9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java @@ -19,9 +19,11 @@ package org.apache.tajo.engine.function.python; import org.apache.hadoop.util.Shell; +import org.apache.tajo.engine.query.QueryContext; import javax.script.ScriptEngine; import java.io.*; +import java.net.URL; import java.util.Map; public abstract class TajoScriptEngine { @@ -91,4 +93,34 @@ private static InputStream getResourceUsingClassLoader(String fullFilename, Clas * @throws java.io.IOException */ protected abstract Map getParamsFromVariables() throws IOException; + +// /** +// * Registers scripting language functions as Pig functions with given namespace +// * +// * @param path path of the script +// * @param namespace namespace for the functions +// * @param context context to register functions to tajo in the given namespace +// * @throws IOException +// */ +// public abstract void registerFunctions(String path, String namespace, +// QueryContext context) throws IOException; + + /** + * Figures out the jar location from the class + * @param clazz class in the jar file + * @return the jar file location, null if the class was not loaded from a jar + * @throws FileNotFoundException + */ + public static String getJarPath(Class clazz) + throws FileNotFoundException { + URL resource = clazz.getClassLoader().getResource( + clazz.getCanonicalName().replace(".", "/") + ".class"); + if (resource.getProtocol().equals("jar")) { + return resource.getPath().substring( + resource.getPath().indexOf(':') + 1, + resource.getPath().indexOf('!')); + } + throw new FileNotFoundException("Jar for " + clazz.getName() + + " class is not found"); + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java index a013dc0ac8..7b3c00db30 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java @@ -166,10 +166,4 @@ public void setInsert() { public boolean isInsert() { return isCommandType(NodeType.INSERT.name()); } - - public void addScriptJar(String modulepath) { - } - - public void addScriptFile(String modulename, String modulepath) { - } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 371dfb4779..04f1194819 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -21,10 +21,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.service.CompositeService; @@ -36,10 +33,14 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.tajo.catalog.CatalogServer; import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.LocalCatalogWrapper; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.engine.function.FunctionLoader; +import org.apache.tajo.engine.function.python.JythonScriptEngine; +import org.apache.tajo.engine.function.python.OptionalFunctionContext; +import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.master.rm.TajoWorkerResourceManager; import org.apache.tajo.master.rm.WorkerResourceManager; import org.apache.tajo.metrics.CatalogMetricsGaugeSet; @@ -67,7 +68,9 @@ import java.lang.reflect.Constructor; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import java.util.Map; import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; import static org.apache.tajo.TajoConstants.DEFAULT_TABLESPACE_NAME; @@ -134,6 +137,9 @@ public class TajoMaster extends CompositeService { private HistoryReader historyReader; + // context for non-builtin functions + private OptionalFunctionContext functionContext = new OptionalFunctionContext(); + public TajoMaster() throws Exception { super(TajoMaster.class.getName()); } @@ -175,7 +181,7 @@ public void serviceInit(Configuration _conf) throws Exception { diagnoseTajoMaster(); this.storeManager = StorageManager.getFileStorageManager(systemConf); - catalogServer = new CatalogServer(FunctionLoader.load()); + catalogServer = new CatalogServer(loadFunctions()); addIfService(catalogServer); catalog = new LocalCatalogWrapper(catalogServer, systemConf); @@ -202,6 +208,11 @@ public void serviceInit(Configuration _conf) throws Exception { LOG.info("Tajo Master is initialized."); } + private Collection loadFunctions() throws IOException { + Map functionMap = FunctionLoader.load(); + return FunctionLoader.loadOptionalFunctions(systemConf, functionContext, functionMap).values(); + } + private void initSystemMetrics() { systemMetrics = new TajoSystemMetrics(systemConf, METRICS_GROUP_NAME, getMasterName()); systemMetrics.start(); @@ -474,6 +485,10 @@ public HistoryWriter getHistoryWriter() { public HistoryReader getHistoryReader() { return historyReader; } + + public OptionalFunctionContext getFunctionContext() { + return functionContext; + } } String getThreadTaskName(long id, String name) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 3c55add775..4966862c32 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -33,7 +33,12 @@ import org.apache.tajo.TajoConstants; import org.apache.tajo.catalog.CatalogClient; import org.apache.tajo.catalog.CatalogService; +import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.function.FunctionLoader; +import org.apache.tajo.engine.function.python.JythonScriptEngine; +import org.apache.tajo.engine.function.python.OptionalFunctionContext; +import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.service.TajoMasterInfo; @@ -51,10 +56,7 @@ import org.apache.tajo.rule.SelfDiagnosisRuleSession; import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.storage.StorageManager; -import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.JvmPauseMonitor; -import org.apache.tajo.util.NetUtils; -import org.apache.tajo.util.StringUtils; +import org.apache.tajo.util.*; import org.apache.tajo.util.history.HistoryReader; import org.apache.tajo.util.history.HistoryWriter; import org.apache.tajo.util.metrics.TajoSystemMetrics; @@ -65,7 +67,9 @@ import java.lang.management.ThreadInfo; import java.lang.management.ThreadMXBean; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -144,6 +148,9 @@ public class TajoWorker extends CompositeService { private HistoryReader historyReader; + // context for non-builtin functions + private OptionalFunctionContext functionContext = new OptionalFunctionContext(); + public TajoWorker() throws Exception { super(TajoWorker.class.getName()); } @@ -278,10 +285,27 @@ public void serviceInit(Configuration conf) throws Exception { taskHistoryWriter.init(conf); historyReader = new HistoryReader(workerContext.getWorkerName(), this.systemConf); + + FunctionLoader.loadOptionalFunctions(systemConf, functionContext, new HashMap()); diagnoseTajoWorker(); } +// private void initOptionalFunctions() throws IOException { +// String[] codePaths = systemConf.getStrings(TajoConf.ConfVars.PYTHON_CODE_DIR.varname); +// for (String codePath : codePaths) { +// Set functions = JythonScriptEngine.registerFunctions(functionContext, codePath, +// FunctionLoader.PYTHON_FUNCTION_NAMESPACE); +// for (FunctionDesc desc : functions) { +// LOG.info("FunctionLoader: " + desc); +// } +//// for (FunctionDesc f : JythonScriptEngine.registerFunctions(functionContext, codePath, +//// FunctionLoader.PYTHON_FUNCTION_NAMESPACE)) { +//// map.put(f.getSignature(), f); +//// } +// } +// } + private void initWorkerMetrics() { workerSystemMetrics = new TajoSystemMetrics(systemConf, "worker", workerContext.getWorkerName()); workerSystemMetrics.start(); diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 17348e1b25..25c2dc63dd 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -165,6 +165,8 @@ void initPropertiesAndConfigs() { // Memory cache termination conf.setIntVar(ConfVars.WORKER_HISTORY_EXPIRE_PERIOD, 1); + conf.setStrings(ConfVars.PYTHON_CODE_DIR.varname, "python/test1.py"); + this.standbyWorkerMode = conf.getVar(ConfVars.RESOURCE_MANAGER_CLASS) .indexOf(TajoWorkerResourceManager.class.getName()) >= 0; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 876e3e4f1a..02e4c552bb 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -34,6 +34,7 @@ import org.apache.tajo.engine.codegen.EvalCodeGenerator; import org.apache.tajo.engine.codegen.TajoClassLoader; import org.apache.tajo.engine.function.FunctionLoader; +import org.apache.tajo.engine.function.python.OptionalFunctionContext; import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.parser.SQLAnalyzer; import org.apache.tajo.plan.*; @@ -91,7 +92,7 @@ public static void setUp() throws Exception { cat = util.getMiniCatalogCluster().getCatalog(); cat.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); cat.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); - for (FunctionDesc funcDesc : FunctionLoader.load()) { + for (FunctionDesc funcDesc : FunctionLoader.load().values()) { cat.createFunction(funcDesc); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java index 820ed249ce..72a9e3279a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java @@ -34,6 +34,11 @@ public void testGetFunction() throws Exception { LOG.info("File path: " + url); PyFunction function = JythonScriptEngine.getFunction(url.getPath(), "return_one"); LOG.info(function.getType()); - function.__call__(); + LOG.info(function.__call__().toString()); + } + + public void testRegisterFunction() throws Exception { + OptionalFunctionContext context = new OptionalFunctionContext(); + JythonScriptEngine.registerFunctions(context, "python/test1.py", "test"); } } \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/__init__.py b/tajo-core/src/test/resources/python/__init__.py index 67bb99de4d..8093a2f8fa 100644 --- a/tajo-core/src/test/resources/python/__init__.py +++ b/tajo-core/src/test/resources/python/__init__.py @@ -1,17 +1,17 @@ -""" -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 +#!/usr/bin/python - 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. -""" \ No newline at end of file +############################################################################ +# 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. \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/tajo_util.py b/tajo-core/src/test/resources/python/tajo_util.py index a505145a52..75204d69dd 100644 --- a/tajo-core/src/test/resources/python/tajo_util.py +++ b/tajo-core/src/test/resources/python/tajo_util.py @@ -1,20 +1,20 @@ -""" -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 +#!/usr/bin/python - 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. -""" +############################################################################ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. import logging diff --git a/tajo-core/src/test/resources/python/test1.py b/tajo-core/src/test/resources/python/test1.py index 47abcddd1f..9cd31646ad 100644 --- a/tajo-core/src/test/resources/python/test1.py +++ b/tajo-core/src/test/resources/python/test1.py @@ -1,23 +1,35 @@ -""" -Licensed to the Apache Software Foundation (ASF) under one -or more contributor license agreements. See the NOTICE file -distributed with this work for additional information -regarding copyright ownership. The ASF licenses this file -to you under the Apache License, Version 2.0 (the -"License"); you may not use this file except in compliance -with the License. You may obtain a copy of the License at +#!/usr/bin/python - http://www.apache.org/licenses/LICENSE-2.0 +############################################################################ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. -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. -""" +# from tajo_util import outputSchema -from tajo_util import outputSchema +# @outputSchema('int4') +# def return_one(): +# return 100 +# +# @outputSchema("text") +# def helloworld(): +# return 'Hello, World' +# +# # No decorator - bytearray +# def concat(str): +# return str+str -@outputSchema('value:int') -def return_one(): - return 1 +@outputSchema('int4') +def sum_py(a,b): + return a+b diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index a1a895900c..b1b10cb166 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -31,6 +31,7 @@ public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { @Override public Datum eval(Schema schema, Tuple tuple) { + return null; } } From 06d4a27e65cc75aec2ed1239cdf2215dcf99c00d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 25 Mar 2015 22:34:40 +0900 Subject: [PATCH 05/55] TAJO-1344 --- tajo-core/pom.xml | 6 --- .../tajo/engine/function/FunctionLoader.java | 4 +- .../org/apache/tajo/master/TajoMaster.java | 3 +- .../org/apache/tajo/worker/TajoWorker.java | 4 +- .../apache/tajo/engine/eval/ExprTestBase.java | 8 +++- .../engine/function/TestMathFunctions.java | 5 +++ .../python/TestJythonScriptEngine.java | 2 + tajo-plan/pom.xml | 5 +++ .../plan/expr/GeneralPythonFunctionEval.java | 24 ++++++++++- .../function}/OptionalFunctionContext.java | 2 +- .../function/python/JythonScriptEngine.java | 3 +- .../plan}/function/python/JythonUtils.java | 43 +++++++++++++++++-- .../function/python/TajoScriptEngine.java | 3 +- 13 files changed, 88 insertions(+), 24 deletions(-) rename {tajo-core/src/main/java/org/apache/tajo/engine/function/python => tajo-plan/src/main/java/org/apache/tajo/plan/function}/OptionalFunctionContext.java (97%) rename {tajo-core/src/main/java/org/apache/tajo/engine => tajo-plan/src/main/java/org/apache/tajo/plan}/function/python/JythonScriptEngine.java (99%) rename {tajo-core/src/main/java/org/apache/tajo/engine => tajo-plan/src/main/java/org/apache/tajo/plan}/function/python/JythonUtils.java (57%) rename {tajo-core/src/main/java/org/apache/tajo/engine => tajo-plan/src/main/java/org/apache/tajo/plan}/function/python/TajoScriptEngine.java (97%) diff --git a/tajo-core/pom.xml b/tajo-core/pom.xml index bc5da039f3..61a156b668 100644 --- a/tajo-core/pom.xml +++ b/tajo-core/pom.xml @@ -454,12 +454,6 @@ jcip-annotations test - - - org.python - jython-standalone - 2.7-b3 - diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index f8f6a778f1..1fe74e32ab 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -36,9 +36,9 @@ import org.apache.tajo.engine.function.annotation.Description; import org.apache.tajo.engine.function.annotation.ParamOptionTypes; import org.apache.tajo.engine.function.annotation.ParamTypes; -import org.apache.tajo.engine.function.python.JythonScriptEngine; -import org.apache.tajo.engine.function.python.OptionalFunctionContext; import org.apache.tajo.function.*; +import org.apache.tajo.plan.function.OptionalFunctionContext; +import org.apache.tajo.plan.function.python.JythonScriptEngine; import org.apache.tajo.util.ClassUtil; import org.apache.tajo.util.TUtil; diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 04f1194819..528644e816 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -38,13 +38,12 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.conf.TajoConf.ConfVars; import org.apache.tajo.engine.function.FunctionLoader; -import org.apache.tajo.engine.function.python.JythonScriptEngine; -import org.apache.tajo.engine.function.python.OptionalFunctionContext; import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.master.rm.TajoWorkerResourceManager; import org.apache.tajo.master.rm.WorkerResourceManager; import org.apache.tajo.metrics.CatalogMetricsGaugeSet; import org.apache.tajo.metrics.WorkerResourceMetricsGaugeSet; +import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.rpc.RpcChannelFactory; import org.apache.tajo.rule.EvaluationContext; import org.apache.tajo.rule.EvaluationFailedException; diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 4966862c32..c393b90a48 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -36,9 +36,8 @@ import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.function.FunctionLoader; -import org.apache.tajo.engine.function.python.JythonScriptEngine; -import org.apache.tajo.engine.function.python.OptionalFunctionContext; import org.apache.tajo.function.FunctionSignature; +import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.service.TajoMasterInfo; @@ -69,7 +68,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; -import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 02e4c552bb..9c0be89351 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -34,11 +34,12 @@ import org.apache.tajo.engine.codegen.EvalCodeGenerator; import org.apache.tajo.engine.codegen.TajoClassLoader; import org.apache.tajo.engine.function.FunctionLoader; -import org.apache.tajo.engine.function.python.OptionalFunctionContext; import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.parser.SQLAnalyzer; +import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.plan.*; import org.apache.tajo.plan.expr.EvalNode; +import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.plan.serder.EvalNodeDeserializer; import org.apache.tajo.plan.serder.EvalNodeSerializer; import org.apache.tajo.engine.query.QueryContext; @@ -59,6 +60,7 @@ import java.io.IOException; import java.util.List; +import java.util.Map; import java.util.TimeZone; import static org.apache.tajo.TajoConstants.DEFAULT_DATABASE_NAME; @@ -92,7 +94,9 @@ public static void setUp() throws Exception { cat = util.getMiniCatalogCluster().getCatalog(); cat.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); cat.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); - for (FunctionDesc funcDesc : FunctionLoader.load().values()) { + Map map = FunctionLoader.load(); + map = FunctionLoader.loadOptionalFunctions(conf, new OptionalFunctionContext(), map); + for (FunctionDesc funcDesc : map.values()) { cat.createFunction(funcDesc); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java index 49ef7e0549..02dcfbaf90 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java @@ -468,4 +468,9 @@ public void testRoundWithSpecifiedPrecision() throws IOException { testEval(schema, "table1", ",", "select round(col1, col2) from table1", new String[]{""}); } + + @Test + public void test() throws IOException { + testSimpleEval("select sum_py(1,2)", new String[]{""}); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java index 72a9e3279a..c8ad916195 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java @@ -21,6 +21,8 @@ import junit.framework.TestCase; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.tajo.plan.function.OptionalFunctionContext; +import org.apache.tajo.plan.function.python.JythonScriptEngine; import org.apache.tajo.util.FileUtil; import org.python.core.PyFunction; diff --git a/tajo-plan/pom.xml b/tajo-plan/pom.xml index f2f00671aa..d0a50e6c83 100644 --- a/tajo-plan/pom.xml +++ b/tajo-plan/pom.xml @@ -185,6 +185,11 @@ mockito-core test + + org.python + jython-standalone + 2.7-b3 + diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index b1b10cb166..cb4fe33e6b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -20,8 +20,16 @@ import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; +import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; +import org.apache.tajo.function.PythonInvocationDesc; +import org.apache.tajo.plan.function.python.JythonScriptEngine; +import org.apache.tajo.plan.function.python.JythonUtils; import org.apache.tajo.storage.Tuple; +import org.python.core.PyFunction; +import org.python.core.PyObject; + +import java.io.IOException; public class GeneralPythonFunctionEval extends FunctionEval { @@ -31,7 +39,21 @@ public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { @Override public Datum eval(Schema schema, Tuple tuple) { - + PythonInvocationDesc desc = funcDesc.getInvocation().getPython(); + try { + PyFunction function = JythonScriptEngine.getFunction(desc.getPath(), desc.getName()); + TajoDataTypes.DataType[] paramTypes = funcDesc.getSignature().getParamTypes(); + PyObject result; + if (tuple.size() == 0 || paramTypes.length == 0) { + result = function.__call__(); + } else { + PyObject[] params = JythonUtils.tupleToPyTuple(tuple).getArray(); + result = function.__call__(params); + } + // TODO: result to datum + } catch (IOException e) { + throw new RuntimeException(e); + } return null; } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/OptionalFunctionContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/OptionalFunctionContext.java similarity index 97% rename from tajo-core/src/main/java/org/apache/tajo/engine/function/python/OptionalFunctionContext.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/OptionalFunctionContext.java index 268149f90a..e812abf7cc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/OptionalFunctionContext.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/OptionalFunctionContext.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.tajo.engine.function.python; +package org.apache.tajo.plan.function; import java.io.File; import java.util.LinkedHashMap; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java similarity index 99% rename from tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 0731f89767..4cf022dcd7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.tajo.engine.function.python; +package org.apache.tajo.plan.function.python; import java.io.BufferedReader; import java.io.File; @@ -42,6 +42,7 @@ import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.function.FunctionSupplement; import org.apache.tajo.function.PythonInvocationDesc; +import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.util.TUtil; import org.python.core.*; import org.python.modules.zipimport.zipimporter; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java similarity index 57% rename from tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonUtils.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java index 05ef6bbbae..956bcb0d0d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/JythonUtils.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java @@ -16,11 +16,13 @@ * limitations under the License. */ -package org.apache.tajo.engine.function.python; +package org.apache.tajo.plan.function.python; -import org.apache.tajo.catalog.Schema; - -import java.util.HashMap; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.storage.Tuple; +import org.python.core.Py; +import org.python.core.PyObject; +import org.python.core.PyTuple; public class JythonUtils { @@ -44,4 +46,37 @@ public class JythonUtils { // LogicalSchema schema = queryParser.parseSchema(schemaString); // return schema; // } + + public static PyObject datumToPyObject(Datum v) { + return Py.java2py(v.asByteArray()); + +// if (object instanceof Tuple) { +// return tupleToPyTuple((Tuple) object); +// } else if (object instanceof DataBag) { +// PyList list = new PyList(); +// for (Tuple bagTuple : (DataBag) object) { +// list.add(tupleToPyTuple(bagTuple)); +// } +// return list; +// } else if (object instanceof Map) { +// PyDictionary newMap = new PyDictionary(); +// for (Map.Entry entry : ((Map) object).entrySet()) { +// newMap.put(entry.getKey(), datumToPyObject(entry.getValue())); +// } +// return newMap; +// } else if (object instanceof DataByteArray) { +// return Py.java2py(((DataByteArray) object).get()); +// } else { +// return Py.java2py(object); +// } + } + + public static PyTuple tupleToPyTuple(Tuple tuple) { + PyObject[] pyTuple = new PyObject[tuple.size()]; + int i = 0; + for (Datum v : tuple.getValues()) { + pyTuple[i++] = datumToPyObject(v); + } + return new PyTuple(pyTuple); + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java similarity index 97% rename from tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index ff382139f9..c52e2a7aa5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -16,10 +16,9 @@ * limitations under the License. */ -package org.apache.tajo.engine.function.python; +package org.apache.tajo.plan.function.python; import org.apache.hadoop.util.Shell; -import org.apache.tajo.engine.query.QueryContext; import javax.script.ScriptEngine; import java.io.*; From 44afbe25f2021b939aa4498a6791e0f268b6d5e7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 26 Mar 2015 15:34:27 +0900 Subject: [PATCH 06/55] TAJO-1344 --- .../org/apache/tajo/catalog/FunctionDesc.java | 4 +- .../tajo/function/FunctionInvocation.java | 8 +- .../tajo/function/PythonInvocationDesc.java | 15 +- .../src/main/proto/CatalogProtos.proto | 6 + .../apache/tajo/catalog/TestFunctionDesc.java | 4 +- .../org/apache/tajo/catalog/TestCatalog.java | 4 +- .../java/org/apache/tajo/datum/AnyDatum.java | 162 ++++++++++++++++++ .../org/apache/tajo/datum/DatumFactory.java | 6 + .../engine/codegen/EvalCodeGenContext.java | 6 +- .../codegen/LegacyFunctionBindingEmitter.java | 4 +- .../engine/codegen/VariablesPreBuilder.java | 2 +- .../org/apache/tajo/plan/ExprAnnotator.java | 6 +- .../plan/expr/GeneralPythonFunctionEval.java | 17 +- 13 files changed, 225 insertions(+), 19 deletions(-) create mode 100644 tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java index 23d39f2528..6ea6ac6138 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/FunctionDesc.java @@ -92,7 +92,7 @@ public FunctionSupplement getSupplement() { */ public Function newInstance() throws InternalException { try { - Constructor cons = getFuncClass().getConstructor(); + Constructor cons = getLegacyFuncClass().getConstructor(); return cons.newInstance(); } catch (Exception ioe) { throw new InternalException("Cannot initiate function " + signature); @@ -124,7 +124,7 @@ public DataType getReturnType() { //////////////////////////////////////// @SuppressWarnings("unchecked") - public Class getFuncClass() { + public Class getLegacyFuncClass() { return invocation.getLegacy().getFunctionClass(); } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java index 3d92a98445..911d5ddefb 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionInvocation.java @@ -57,7 +57,9 @@ public FunctionInvocation(FunctionInvocationProto proto) { if (proto.hasAggregationJIT()) { this.aggregationJIT = new ClassBaseInvocationDesc(proto.getAggregation()); } - // TODO: add PythonInvocationDesc to FucntionInvocationProto + if (proto.hasPython()) { + this.python = new PythonInvocationDesc(proto.getPython()); + } } public boolean isAvailable() { @@ -154,7 +156,9 @@ public FunctionInvocationProto getProto() { if (hasAggregationJIT()) { builder.setAggregationJIT(aggregationJIT.getProto()); } - // TODO: add PythonInvocationDesc to FucntionInvocationProto + if (hasPython()) { + builder.setPython(python.getProto()); + } return builder.build(); } diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java index 9df81e1f5e..04e70d7e0a 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java @@ -18,7 +18,10 @@ package org.apache.tajo.function; -public class PythonInvocationDesc { +import org.apache.tajo.catalog.proto.CatalogProtos.PythonInvocationDescProto; +import org.apache.tajo.common.ProtoObject; + +public class PythonInvocationDesc implements ProtoObject { private final String funcName; private final String filePath; @@ -27,6 +30,10 @@ public PythonInvocationDesc(String funcName, String filePath) { this.filePath = filePath; } + public PythonInvocationDesc(PythonInvocationDescProto proto) { + this(proto.getFuncName(), proto.getFilePath()); + } + public String getName() { return funcName; } @@ -34,4 +41,10 @@ public String getName() { public String getPath() { return filePath; } + + @Override + public PythonInvocationDescProto getProto() { + return PythonInvocationDescProto.newBuilder() + .setFuncName(funcName).setFilePath(filePath).build(); + } } diff --git a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto index a204685f63..e1a2ab89b6 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto +++ b/tajo-catalog/tajo-catalog-common/src/main/proto/CatalogProtos.proto @@ -397,6 +397,7 @@ message FunctionInvocationProto { optional ClassBaseInvocationDescProto aggregation = 3; optional StaticMethodInvocationDescProto scalarJIT = 4; optional ClassBaseInvocationDescProto aggregationJIT = 5; + optional PythonInvocationDescProto python = 6; } message ClassBaseInvocationDescProto { @@ -409,3 +410,8 @@ message StaticMethodInvocationDescProto { required string returnClass = 3; repeated string paramClasses = 4; } + +message PythonInvocationDescProto { + required string funcName = 1; + required string filePath = 2; +} \ No newline at end of file diff --git a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java index 92d2aa43e0..4a67ce6b4d 100644 --- a/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/test/java/org/apache/tajo/catalog/TestFunctionDesc.java @@ -68,7 +68,7 @@ public void testGetSignature() throws IOException, ClassNotFoundException { desc.setDetail("detail"); assertEquals("sum", desc.getFunctionName()); - assertEquals(TestSum.class, desc.getFuncClass()); + assertEquals(TestSum.class, desc.getLegacyFuncClass()); assertEquals(FunctionType.GENERAL, desc.getFuncType()); assertEquals(Type.INT4, desc.getReturnType().getType()); assertArrayEquals(CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.INT8), @@ -84,7 +84,7 @@ public void testGetSignature() throws IOException, ClassNotFoundException { FunctionDesc newDesc = new FunctionDesc(proto); assertEquals("sum", newDesc.getFunctionName()); - assertEquals(TestSum.class, newDesc.getFuncClass()); + assertEquals(TestSum.class, newDesc.getLegacyFuncClass()); assertEquals(FunctionType.GENERAL, newDesc.getFuncType()); assertEquals(Type.INT4, newDesc.getReturnType().getType()); diff --git a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java index c3bfc99bb0..98876a7a0d 100644 --- a/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java +++ b/tajo-catalog/tajo-catalog-server/src/test/java/org/apache/tajo/catalog/TestCatalog.java @@ -664,7 +664,7 @@ public final void testRegisterAndFindFunc() throws Exception { FunctionDesc retrived = catalog.getFunction("test10", CatalogUtil.newSimpleDataTypeArray(Type.INT4, Type.BLOB)); assertEquals(retrived.getFunctionName(), "test10"); - assertEquals(retrived.getFuncClass(), TestFunc2.class); + assertEquals(retrived.getLegacyFuncClass(), TestFunc2.class); assertEquals(retrived.getFuncType(), FunctionType.GENERAL); assertFalse(catalog.containFunction("test10", CatalogUtil.newSimpleDataTypeArray(Type.BLOB, Type.INT4))); @@ -683,7 +683,7 @@ public final void testRegisterFunc() throws Exception { FunctionDesc retrived = catalog.getFunction("test2", CatalogUtil.newSimpleDataTypeArray(Type.INT4)); assertEquals(retrived.getFunctionName(),"test2"); - assertEquals(retrived.getFuncClass(),TestFunc1.class); + assertEquals(retrived.getLegacyFuncClass(),TestFunc1.class); assertEquals(retrived.getFuncType(),FunctionType.UDF); } diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java new file mode 100644 index 0000000000..025fbde8b4 --- /dev/null +++ b/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.datum; + +import com.google.gson.annotations.Expose; +import org.apache.tajo.exception.InvalidOperationException; + +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.util.Arrays; + +import static org.apache.tajo.common.TajoDataTypes.Type.ANY; + +public class AnyDatum extends Datum { + @Expose + private final byte [] val; + private ByteBuffer bb = null; + + public AnyDatum(byte[] val) { + super(ANY); + this.val = val; + this.bb = ByteBuffer.wrap(val); + bb.flip(); + } + + public AnyDatum(byte[] val, int offset, int length) { + super(ANY); + byte[] b = new byte[length]; + System.arraycopy(val, offset, b, 0 , length); + this.val = b; + this.bb = ByteBuffer.wrap(b); + bb.flip(); + } + + public AnyDatum(ByteBuffer val) { + super(ANY); + this.val = val.array(); + this.bb = val.duplicate(); + bb.flip(); + } + + public void initFromBytes() { + if (bb == null) { + bb = ByteBuffer.wrap(val); + } + } + + @Override + public int asInt4() { + initFromBytes(); + bb.rewind(); + return bb.getInt(); + } + + @Override + public long asInt8() { + initFromBytes(); + bb.rewind(); + return bb.getLong(); + } + + @Override + public byte asByte() { + initFromBytes(); + bb.rewind(); + return bb.get(); + } + + @Override + public byte[] asByteArray() { + initFromBytes(); + bb.rewind(); + return bb.array(); + } + + @Override + public float asFloat4() { + initFromBytes(); + bb.rewind(); + return bb.getFloat(); + } + + @Override + public double asFloat8() { + initFromBytes(); + bb.rewind(); + return bb.getDouble(); + } + + @Override + public String asChars() { + initFromBytes(); + bb.rewind(); + return new String(bb.array(), Charset.defaultCharset()); + } + + @Override + public int size() { + return this.val.length; + } + + @Override + public int hashCode() { + initFromBytes(); + bb.rewind(); + return bb.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof AnyDatum) { + AnyDatum other = (AnyDatum) obj; + initFromBytes(); + other.initFromBytes(); + return bb.equals(other.bb); + } + + return false; + } + + @Override + public Datum equalsTo(Datum datum) { + switch (datum.type()) { + case BLOB: + return DatumFactory.createBool(Arrays.equals(this.val, ((AnyDatum) datum).val)); + case NULL_TYPE: + return datum; + default: + throw new InvalidOperationException(datum.type()); + } + } + + @Override + public int compareTo(Datum datum) { + switch (datum.type()) { + case BLOB: + initFromBytes(); + ((AnyDatum)datum).initFromBytes(); + return bb.compareTo(((AnyDatum) datum).bb); + case NULL_TYPE: + return -1; + default: + throw new InvalidOperationException(datum.type()); + } + } +} diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java index 9f48cadc81..318f113e51 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java @@ -405,6 +405,10 @@ public static Inet4Datum createInet4(String val) { return new Inet4Datum(val); } + public static AnyDatum createAny(byte[] val) { + return new AnyDatum(val); + } + public static Datum cast(Datum operandDatum, DataType target, @Nullable TimeZone tz) { switch (target.getType()) { case BOOLEAN: @@ -454,6 +458,8 @@ public static Datum cast(Datum operandDatum, DataType target, @Nullable TimeZone return DatumFactory.createBlob(operandDatum.asByteArray()); case INET4: return DatumFactory.createInet4(operandDatum.asByteArray()); + case ANY: + return DatumFactory.createAny(operandDatum.asByteArray()); default: throw new InvalidCastException(operandDatum.type(), target.getType()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java index 8384de71ec..464488b066 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/EvalCodeGenContext.java @@ -174,7 +174,7 @@ public void emitConstructor() { } else if (entry.getKey().getType() == EvalType.FUNCTION) { GeneralFunctionEval function = (GeneralFunctionEval) entry.getKey(); - final String internalName = TajoGeneratorAdapter.getInternalName(function.getFuncDesc().getFuncClass()); + final String internalName = TajoGeneratorAdapter.getInternalName(function.getFuncDesc().getLegacyFuncClass()); // new and initialization of function initMethod.visitTypeInsn(Opcodes.NEW, internalName); @@ -198,12 +198,12 @@ public void emitConstructor() { initMethod.visitVarInsn(Opcodes.ALOAD, FUNCTION); consAdapter.aload(PARAM_TYPE_ARRAY); - consAdapter.invokeVirtual(function.getFuncDesc().getFuncClass(), "init", void.class, new Class[] {FunctionEval.ParamType[].class}); + consAdapter.invokeVirtual(function.getFuncDesc().getLegacyFuncClass(), "init", void.class, new Class[] {FunctionEval.ParamType[].class}); initMethod.visitVarInsn(Opcodes.ALOAD, 0); initMethod.visitVarInsn(Opcodes.ALOAD, FUNCTION); initMethod.visitFieldInsn(Opcodes.PUTFIELD, this.owner, entry.getValue(), - "L" + TajoGeneratorAdapter.getInternalName(function.getFuncDesc().getFuncClass()) + ";"); + "L" + TajoGeneratorAdapter.getInternalName(function.getFuncDesc().getLegacyFuncClass()) + ";"); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/LegacyFunctionBindingEmitter.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/LegacyFunctionBindingEmitter.java index 36dfe4fb13..6b47149d20 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/LegacyFunctionBindingEmitter.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/LegacyFunctionBindingEmitter.java @@ -63,12 +63,12 @@ public static void emit(EvalCodeGenerator generator, EvalCodeGenContext context, FunctionDesc desc = func.getFuncDesc(); String fieldName = context.symbols.get(func); - String funcDescName = "L" + TajoGeneratorAdapter.getInternalName(desc.getFuncClass()) + ";"; + String funcDescName = "L" + TajoGeneratorAdapter.getInternalName(desc.getLegacyFuncClass()) + ";"; context.aload(0); context.methodvisitor.visitFieldInsn(Opcodes.GETFIELD, context.owner, fieldName, funcDescName); context.aload(TUPLE); - context.invokeVirtual(desc.getFuncClass(), "eval", Datum.class, new Class[] {Tuple.class}); + context.invokeVirtual(desc.getLegacyFuncClass(), "eval", Datum.class, new Class[] {Tuple.class}); context.convertToPrimitive(func.getValueType()); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java index a055b049a4..95ec37105f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/codegen/VariablesPreBuilder.java @@ -75,7 +75,7 @@ public EvalNode visitFuncCall(EvalCodeGenContext context, FunctionEval function, String fieldName = function.getFuncDesc().getFunctionName() + "_" + context.seqId++; context.symbols.put(function, fieldName); context.classWriter.visitField(Opcodes.ACC_PRIVATE, fieldName, - "L" + TajoGeneratorAdapter.getInternalName(function.getFuncDesc().getFuncClass()) + ";", null, null); + "L" + TajoGeneratorAdapter.getInternalName(function.getFuncDesc().getLegacyFuncClass()) + ";", null, null); } return function; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index 127d5bd232..1c88ea77dc 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -608,7 +608,11 @@ public EvalNode visitFunction(Context ctx, Stack stack, FunctionExpr expr) FunctionType functionType = funcDesc.getFuncType(); if (functionType == FunctionType.GENERAL || functionType == FunctionType.UDF) { - return new GeneralFunctionEval(ctx.queryContext, funcDesc, (GeneralFunction) funcDesc.newInstance(), givenArgs); + if (funcDesc.getInvocation().hasPython()) { + return new GeneralPythonFunctionEval(funcDesc, givenArgs); + } else { + return new GeneralFunctionEval(ctx.queryContext, funcDesc, (GeneralFunction) funcDesc.newInstance(), givenArgs); + } } else if (functionType == FunctionType.AGGREGATION || functionType == FunctionType.UDA) { if (!ctx.currentBlock.hasNode(NodeType.GROUP_BY)) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index cb4fe33e6b..99b2417918 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -26,12 +26,14 @@ import org.apache.tajo.plan.function.python.JythonScriptEngine; import org.apache.tajo.plan.function.python.JythonUtils; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; import org.python.core.PyFunction; import org.python.core.PyObject; import java.io.IOException; public class GeneralPythonFunctionEval extends FunctionEval { + private Tuple params = null; public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { super(EvalType.FUNCTION, funcDesc, argEvals); @@ -39,16 +41,25 @@ public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { @Override public Datum eval(Schema schema, Tuple tuple) { + if (this.params == null) { + params = new VTuple(argEvals.length); + } + if(argEvals != null) { + params.clear(); + for(int i=0;i < argEvals.length; i++) { + params.put(i, argEvals[i].eval(schema, tuple)); + } + } PythonInvocationDesc desc = funcDesc.getInvocation().getPython(); try { PyFunction function = JythonScriptEngine.getFunction(desc.getPath(), desc.getName()); TajoDataTypes.DataType[] paramTypes = funcDesc.getSignature().getParamTypes(); PyObject result; - if (tuple.size() == 0 || paramTypes.length == 0) { + if (paramTypes.length == 0) { result = function.__call__(); } else { - PyObject[] params = JythonUtils.tupleToPyTuple(tuple).getArray(); - result = function.__call__(params); + PyObject[] pyParams = JythonUtils.tupleToPyTuple(params).getArray(); + result = function.__call__(pyParams); } // TODO: result to datum } catch (IOException e) { From dae49bdbd6d13ff1503168591260569863683e3f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 26 Mar 2015 19:00:52 +0900 Subject: [PATCH 07/55] TAJO-1344 --- .../org/apache/tajo/catalog/CatalogUtil.java | 3 +- .../tajo/function/PythonInvocationDesc.java | 17 ++- .../java/org/apache/tajo/datum/AnyDatum.java | 124 +++--------------- .../org/apache/tajo/datum/DatumFactory.java | 6 +- .../org/apache/tajo/json/DatumAdapter.java | 7 +- .../engine/function/TestMathFunctions.java | 16 ++- .../tajo/engine/query/TestSelectQuery.java | 6 + tajo-core/src/test/resources/python/test1.py | 22 ++-- .../tajo/plan/expr/BasicEvalNodeVisitor.java | 8 ++ .../tajo/plan/expr/EvalNodeVisitor2.java | 1 + .../org/apache/tajo/plan/expr/EvalType.java | 2 + .../plan/expr/GeneralPythonFunctionEval.java | 6 +- .../tajo/plan/expr/SimpleEvalNodeVisitor.java | 3 + .../exprrewrite/rules/ConstantFolding.java | 4 +- .../plan/function/python/JythonUtils.java | 80 ++++++++++- .../plan/serder/EvalNodeDeserializer.java | 4 + .../tajo/plan/serder/EvalNodeSerializer.java | 10 +- tajo-plan/src/main/proto/Plan.proto | 28 ++-- 18 files changed, 196 insertions(+), 151 deletions(-) diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java index afcff2d36f..e8b7ea4164 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/catalog/CatalogUtil.java @@ -24,7 +24,6 @@ import org.apache.tajo.TajoConstants; import org.apache.tajo.catalog.partition.PartitionMethodDesc; import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto; import org.apache.tajo.catalog.proto.CatalogProtos.SchemaProto; import org.apache.tajo.catalog.proto.CatalogProtos.TableDescProto; import org.apache.tajo.common.TajoDataTypes; @@ -418,7 +417,7 @@ public static boolean isMatchedFunction(List definedTypes, List { - private final String funcName; - private final String filePath; + @Expose private String funcName; + @Expose private String filePath; + + public PythonInvocationDesc() { + + } public PythonInvocationDesc(String funcName, String filePath) { this.funcName = funcName; this.filePath = filePath; } + public void setFuncName(String funcName) { + this.funcName = funcName; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } + public PythonInvocationDesc(PythonInvocationDescProto proto) { this(proto.getFuncName(), proto.getFilePath()); } diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java index 025fbde8b4..b0deb3a446 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java @@ -19,144 +19,60 @@ package org.apache.tajo.datum; import com.google.gson.annotations.Expose; -import org.apache.tajo.exception.InvalidOperationException; - -import java.nio.ByteBuffer; -import java.nio.charset.Charset; -import java.util.Arrays; import static org.apache.tajo.common.TajoDataTypes.Type.ANY; public class AnyDatum extends Datum { - @Expose - private final byte [] val; - private ByteBuffer bb = null; + @Expose Datum val; - public AnyDatum(byte[] val) { + public AnyDatum(Datum val) { super(ANY); this.val = val; - this.bb = ByteBuffer.wrap(val); - bb.flip(); } - public AnyDatum(byte[] val, int offset, int length) { - super(ANY); - byte[] b = new byte[length]; - System.arraycopy(val, offset, b, 0 , length); - this.val = b; - this.bb = ByteBuffer.wrap(b); - bb.flip(); - } - - public AnyDatum(ByteBuffer val) { - super(ANY); - this.val = val.array(); - this.bb = val.duplicate(); - bb.flip(); - } - - public void initFromBytes() { - if (bb == null) { - bb = ByteBuffer.wrap(val); - } - } - - @Override - public int asInt4() { - initFromBytes(); - bb.rewind(); - return bb.getInt(); - } - - @Override - public long asInt8() { - initFromBytes(); - bb.rewind(); - return bb.getLong(); - } - - @Override - public byte asByte() { - initFromBytes(); - bb.rewind(); - return bb.get(); - } - - @Override - public byte[] asByteArray() { - initFromBytes(); - bb.rewind(); - return bb.array(); - } - - @Override - public float asFloat4() { - initFromBytes(); - bb.rewind(); - return bb.getFloat(); - } - - @Override - public double asFloat8() { - initFromBytes(); - bb.rewind(); - return bb.getDouble(); - } - - @Override - public String asChars() { - initFromBytes(); - bb.rewind(); - return new String(bb.array(), Charset.defaultCharset()); + public Datum getActual() { + return val; } @Override public int size() { - return this.val.length; + return this.val.size(); } @Override public int hashCode() { - initFromBytes(); - bb.rewind(); - return bb.hashCode(); + return val.hashCode(); } @Override public boolean equals(Object obj) { if (obj instanceof AnyDatum) { AnyDatum other = (AnyDatum) obj; - initFromBytes(); - other.initFromBytes(); - return bb.equals(other.bb); + return val.equals(other.val); } - return false; } @Override public Datum equalsTo(Datum datum) { - switch (datum.type()) { - case BLOB: - return DatumFactory.createBool(Arrays.equals(this.val, ((AnyDatum) datum).val)); - case NULL_TYPE: - return datum; - default: - throw new InvalidOperationException(datum.type()); + if (datum.type() == ANY) { + AnyDatum other = (AnyDatum) datum; + return val.equalsTo(other.val); } + return DatumFactory.createBool(false); } @Override public int compareTo(Datum datum) { - switch (datum.type()) { - case BLOB: - initFromBytes(); - ((AnyDatum)datum).initFromBytes(); - return bb.compareTo(((AnyDatum) datum).bb); - case NULL_TYPE: - return -1; - default: - throw new InvalidOperationException(datum.type()); + if (datum.type() == ANY) { + AnyDatum other = (AnyDatum) datum; + return val.compareTo(other.val); } + return -1; + } + + @Override + public String toString() { + return val.toString(); } } diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java index 318f113e51..20a2d8962d 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java @@ -66,7 +66,7 @@ public static Class getDatumClass(Type type) { case INET4: return Inet4Datum.class; case ANY: - return NullDatum.class; + return AnyDatum.class; case NULL_TYPE: return NullDatum.class; default: @@ -405,7 +405,7 @@ public static Inet4Datum createInet4(String val) { return new Inet4Datum(val); } - public static AnyDatum createAny(byte[] val) { + public static AnyDatum createAny(Datum val) { return new AnyDatum(val); } @@ -459,7 +459,7 @@ public static Datum cast(Datum operandDatum, DataType target, @Nullable TimeZone case INET4: return DatumFactory.createInet4(operandDatum.asByteArray()); case ANY: - return DatumFactory.createAny(operandDatum.asByteArray()); + return DatumFactory.createAny(operandDatum); default: throw new InvalidCastException(operandDatum.type(), target.getType()); } diff --git a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java b/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java index d65559d7e3..4a2533abf8 100644 --- a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java +++ b/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java @@ -41,8 +41,10 @@ public Datum deserialize(JsonElement json, Type typeOfT, return new TimestampDatum(CommonGsonHelper.getOrDie(jsonObject, "value").getAsLong()); case INTERVAL: String[] values = CommonGsonHelper.getOrDie(jsonObject, "value").getAsString().split(","); - return new IntervalDatum(Integer.parseInt(values[0]), Long.parseLong(values[1])); + case ANY: + return new AnyDatum(context.deserialize(CommonGsonHelper.getOrDie(jsonObject, "actual"), + DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName)))); default: return context.deserialize(CommonGsonHelper.getOrDie(jsonObject, "body"), DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName))); @@ -67,6 +69,9 @@ public JsonElement serialize(Datum src, Type typeOfSrc, JsonSerializationContext IntervalDatum interval = (IntervalDatum)src; jsonObj.addProperty("value", interval.getMonths() + "," + interval.getMilliSeconds()); break; + case ANY: + jsonObj.add("actual", context.serialize(((AnyDatum) src).getActual())); + break; default: jsonObj.add("body", context.serialize(src)); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java index 02dcfbaf90..6a3cec708c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java @@ -394,9 +394,9 @@ public void testDegrees() throws IOException { @Test public void testPow() throws IOException { - testSimpleEval("select pow(9,3) as col1 ", new String[]{String.valueOf(Math.pow(9,3))}); - testSimpleEval("select pow(1.0,3) as col1 ", new String[]{String.valueOf(Math.pow(1.0,3))}); - testSimpleEval("select pow(20.1,3.1) as col1 ", new String[]{String.valueOf(Math.pow(20.1,3.1))}); + testSimpleEval("select pow(9,3) as col1 ", new String[]{String.valueOf(Math.pow(9, 3))}); + testSimpleEval("select pow(1.0,3) as col1 ", new String[]{String.valueOf(Math.pow(1.0, 3))}); + testSimpleEval("select pow(20.1,3.1) as col1 ", new String[]{String.valueOf(Math.pow(20.1, 3.1))}); testSimpleEval("select pow(null,3.1) as col1 ", new String[]{""}); testSimpleEval("select pow(20.1,null) as col1 ", new String[]{""}); @@ -408,8 +408,8 @@ public void testPow() throws IOException { testEval(schema, "table1", "0.4,2.7,3,2", "select pow(col1, col2), pow(col3, col4) from table1", new String[]{ - String.valueOf(Math.pow((float)0.4, 2.7)), - String.valueOf(Math.pow(3,2)) + String.valueOf(Math.pow((float) 0.4, 2.7)), + String.valueOf(Math.pow(3, 2)) }); } @@ -471,6 +471,10 @@ public void testRoundWithSpecifiedPrecision() throws IOException { @Test public void test() throws IOException { - testSimpleEval("select sum_py(1,2)", new String[]{""}); +// testSimpleEval("select pow(1,2)", new String[]{"1.0"}); +// testSimpleEval("select sum_py(1,2)", new String[]{"3"}); +// testSimpleEval("select return_one()", new String[]{"100"}); +// testSimpleEval("select helloworld()", new String[]{"Hello, World"}); + testSimpleEval("select concat_py('1')", new String[]{"11"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 176d44efcd..06203cb723 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -708,4 +708,10 @@ public void testMultiBytesDelimiter4() throws Exception { executeString("DROP TABLE table2"); } } + + @Test + public void test() throws Exception { + ResultSet res = executeString("select concat_py(n_name) from nation where n_nationkey = 1"); + System.out.println(resultSetToString(res)); + } } \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/test1.py b/tajo-core/src/test/resources/python/test1.py index 9cd31646ad..8eb16169b9 100644 --- a/tajo-core/src/test/resources/python/test1.py +++ b/tajo-core/src/test/resources/python/test1.py @@ -18,17 +18,17 @@ # from tajo_util import outputSchema -# @outputSchema('int4') -# def return_one(): -# return 100 -# -# @outputSchema("text") -# def helloworld(): -# return 'Hello, World' -# -# # No decorator - bytearray -# def concat(str): -# return str+str +@outputSchema('int4') +def return_one(): + return 100 + +@outputSchema("text") +def helloworld(): + return 'Hello, World' + +# No decorator - bytearray +def concat_py(str): + return str+str @outputSchema('int4') def sum_py(a,b): diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java index 81b0f8e9dc..4620698429 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java @@ -128,6 +128,9 @@ public RESULT visitChild(CONTEXT context, EvalNode evalNode, Stack sta case WINDOW_FUNCTION: result = visitWindowFunc(context, (WindowFunctionEval) evalNode, stack); break; + case PYTHON_FUNCTION: + result = visitPythonFunc(context, (GeneralPythonFunctionEval) evalNode, stack); + break; case SIGNED: result = visitSigned(context, (SignedEval) evalNode, stack); @@ -333,6 +336,11 @@ public RESULT visitWindowFunc(CONTEXT context, WindowFunctionEval evalNode, Stac return visitDefaultFunctionEval(context, evalNode, stack); } + @Override + public RESULT visitPythonFunc(CONTEXT context, GeneralPythonFunctionEval evalNode, Stack stack) { + return visitDefaultFunctionEval(context, evalNode, stack); + } + @Override public RESULT visitSigned(CONTEXT context, SignedEval signedEval, Stack stack) { return visitDefaultUnaryEval(context, signedEval, stack); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java index bae193abde..99a66d7f77 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java @@ -65,6 +65,7 @@ public interface EvalNodeVisitor2 { RESULT visitFuncCall(CONTEXT context, GeneralFunctionEval evalNode, Stack stack); RESULT visitAggrFuncCall(CONTEXT context, AggregationFunctionCallEval evalNode, Stack stack); RESULT visitWindowFunc(CONTEXT context, WindowFunctionEval evalNode, Stack stack); + RESULT visitPythonFunc(CONTEXT context, GeneralPythonFunctionEval evalNode, Stack stack); RESULT visitSigned(CONTEXT context, SignedEval signedEval, Stack stack); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java index c1df658a1b..5d51d47d5f 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java @@ -47,6 +47,7 @@ public enum EvalType { WINDOW_FUNCTION(WindowFunctionEval.class), AGG_FUNCTION(AggregationFunctionCallEval.class), FUNCTION(GeneralFunctionEval.class), + PYTHON_FUNCTION(GeneralPythonFunctionEval.class), // String operator or pattern matching predicates LIKE(LikePredicateEval.class), @@ -148,6 +149,7 @@ public static boolean isFunction(EvalType type) { match |= type == FUNCTION; match |= type == AGG_FUNCTION; match |= type == WINDOW_FUNCTION; + match |= type == PYTHON_FUNCTION; return match; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index 99b2417918..0d22a8975d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -36,7 +36,7 @@ public class GeneralPythonFunctionEval extends FunctionEval { private Tuple params = null; public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { - super(EvalType.FUNCTION, funcDesc, argEvals); + super(EvalType.PYTHON_FUNCTION, funcDesc, argEvals); } @Override @@ -61,10 +61,10 @@ public Datum eval(Schema schema, Tuple tuple) { PyObject[] pyParams = JythonUtils.tupleToPyTuple(params).getArray(); result = function.__call__(pyParams); } - // TODO: result to datum + + return JythonUtils.pyObjectToDatum(result); } catch (IOException e) { throw new RuntimeException(e); } - return null; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java index e706391c38..d0577587e5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java @@ -74,6 +74,9 @@ public EvalNode visit(CONTEXT context, EvalNode evalNode, Stack stack) case WINDOW_FUNCTION: result = visitFuncCall(context, (FunctionEval) evalNode, stack); break; + case PYTHON_FUNCTION: + result = visitFuncCall(context, (FunctionEval) evalNode, stack); + break; default: throw new UnsupportedException("Unknown EvalType: " + evalNode); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index 0e115e290a..915bc10605 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -29,6 +29,8 @@ public class ConstantFolding extends SimpleEvalNodeVisitor implements EvalTreeOptimizationRule { + private final static String SLEEP_FUNCTION_NAME = "sleep"; + @Override public EvalNode optimize(LogicalPlanner.PlanContext context, EvalNode evalNode) { return visit(context, evalNode, new Stack()); @@ -72,7 +74,7 @@ public EvalNode visitUnaryEval(LogicalPlanner.PlanContext context, Stack stack) { boolean constantOfAllDescendents = true; - if ("sleep".equals(evalNode.getFuncDesc().getFunctionName())) { + if (SLEEP_FUNCTION_NAME.equals(evalNode.getFuncDesc().getFunctionName())) { constantOfAllDescendents = false; } else { if (evalNode.getArgs() != null) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java index 956bcb0d0d..9298be43f5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java @@ -18,11 +18,14 @@ package org.apache.tajo.plan.function.python; +import com.google.common.base.Preconditions; +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.AnyDatum; import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.DatumFactory; +import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.storage.Tuple; -import org.python.core.Py; -import org.python.core.PyObject; -import org.python.core.PyTuple; +import org.python.core.*; public class JythonUtils { @@ -48,7 +51,44 @@ public class JythonUtils { // } public static PyObject datumToPyObject(Datum v) { - return Py.java2py(v.asByteArray()); + Preconditions.checkArgument(v.type() == TajoDataTypes.Type.ANY); + Datum actual = ((AnyDatum) v).getActual(); + switch (actual.type()) { + case NULL_TYPE: + return Py.java2py(null); + case BOOLEAN: + return Py.java2py(actual.asBool()); + case UINT1: + case INT1: + return Py.java2py(actual.asInt2()); + case UINT2: + case INT2: + return Py.java2py(actual.asInt2()); + case UINT4: + case INT4: + return Py.java2py(actual.asInt4()); + case UINT8: + case INT8: + return Py.java2py(actual.asInt8()); + case FLOAT4: + case FLOAT8: + return Py.java2py(actual.asFloat8()); + case CHAR: + case VARCHAR: + case TEXT: + return Py.java2py(actual.asChars()); + case NCHAR: + case NVARCHAR: + return Py.java2py(actual.asUnicodeChars()); + case BLOB: + return Py.java2py(actual.asByteArray()); + case INET4: + return Py.java2py(actual.asByteArray()); + case INET6: + return Py.java2py(actual.asByteArray()); + default: + throw new UnsupportedException("Unsupported type: " + actual.type()); + } // if (object instanceof Tuple) { // return tupleToPyTuple((Tuple) object); @@ -79,4 +119,36 @@ public static PyTuple tupleToPyTuple(Tuple tuple) { } return new PyTuple(pyTuple); } + + public static Datum pyObjectToDatum(PyObject object) { + if (object instanceof PyLong) { + return DatumFactory.createInt8((Long) object.__tojava__(Long.class)); + } else if (object instanceof PyBoolean) { + return DatumFactory.createBool((Boolean) object.__tojava__(Boolean.class)); + } else if (object instanceof PyInteger) { + return DatumFactory.createInt4((Integer) object.__tojava__(Integer.class)); + } else if (object instanceof PyFloat) { + // J(P)ython is loosely typed, supports only float type, + // hence we convert everything to double to save precision + return DatumFactory.createFloat8((Double) object.__tojava__(Double.class)); + } else if (object instanceof PyString) { + return DatumFactory.createText((String) object.__tojava__(String.class)); + } else if (object instanceof PyNone) { + return DatumFactory.createNullDatum(); + } else if (object instanceof PyTuple) { + throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); + } else if (object instanceof PyList) { + throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); + } else if (object instanceof PyDictionary) { + throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); + } else { + Object javaObj = object.__tojava__(byte[].class); + if(javaObj instanceof byte[]) { + return DatumFactory.createBlob((byte[]) javaObj); + } + else { + throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); + } + } + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 5a96054b1f..892b2941af 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -211,6 +211,8 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { current = winFunc; } + } else if (type == EvalType.PYTHON_FUNCTION) { + current = new GeneralPythonFunctionEval(new FunctionDesc(funcProto.getFuncion()), params); } } catch (ClassNotFoundException cnfe) { String functionName = "Unknown"; @@ -309,6 +311,8 @@ public static Datum deserialize(PlanProto.Datum datum) { return new IntervalDatum(datum.getInterval().getMonth(), datum.getInterval().getMsec()); case NULL_TYPE: return NullDatum.get(); + case ANY: + return DatumFactory.createAny(deserialize(datum.getActual())); default: throw new RuntimeException("Unknown data type: " + datum.getType().name()); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java index c7702c5117..e47d620313 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeSerializer.java @@ -24,6 +24,7 @@ import org.apache.tajo.algebra.WindowSpec.WindowFrameEndBoundType; import org.apache.tajo.algebra.WindowSpec.WindowFrameStartBoundType; import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.datum.AnyDatum; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.IntervalDatum; import org.apache.tajo.plan.expr.*; @@ -181,6 +182,7 @@ public EvalNode visitRowConstant(EvalTreeProtoBuilderContext context, RowConstan return rowConst; } + @Override public EvalNode visitField(EvalTreeProtoBuilderContext context, Stack stack, FieldEval field) { PlanProto.EvalNode.Builder builder = createEvalBuilder(context, field); builder.setField(field.getColumnRef().getProto()); @@ -188,6 +190,7 @@ public EvalNode visitField(EvalTreeProtoBuilderContext context, Stack return field; } + @Override public EvalNode visitBetween(EvalTreeProtoBuilderContext context, BetweenPredicateEval between, Stack stack) { // visiting and registering childs @@ -211,6 +214,7 @@ public EvalNode visitBetween(EvalTreeProtoBuilderContext context, BetweenPredica return between; } + @Override public EvalNode visitCaseWhen(EvalTreeProtoBuilderContext context, CaseWhenEval caseWhen, Stack stack) { // visiting and registering childs super.visitCaseWhen(context, caseWhen, stack); @@ -235,6 +239,7 @@ public EvalNode visitCaseWhen(EvalTreeProtoBuilderContext context, CaseWhenEval return caseWhen; } + @Override public EvalNode visitIfThen(EvalTreeProtoBuilderContext context, CaseWhenEval.IfThenEval ifCond, Stack stack) { // visiting and registering childs @@ -254,6 +259,7 @@ public EvalNode visitIfThen(EvalTreeProtoBuilderContext context, CaseWhenEval.If return ifCond; } + @Override public EvalNode visitFuncCall(EvalTreeProtoBuilderContext context, FunctionEval function, Stack stack) { // visiting and registering childs super.visitFuncCall(context, function, stack); @@ -297,7 +303,6 @@ public EvalNode visitFuncCall(EvalTreeProtoBuilderContext context, FunctionEval builder.setWinFunction(windowFuncBuilder); } - context.treeBuilder.addNodes(builder); return function; } @@ -388,6 +393,9 @@ public static PlanProto.Datum serialize(Datum datum) { intervalBuilder.setMsec(interval.getMilliSeconds()); builder.setInterval(intervalBuilder); break; + case ANY: + builder.setActual(serialize(((AnyDatum)datum).getActual())); + break; default: throw new RuntimeException("Unknown data type: " + datum.type().name()); } diff --git a/tajo-plan/src/main/proto/Plan.proto b/tajo-plan/src/main/proto/Plan.proto index 02f52ff626..1dffc229b0 100644 --- a/tajo-plan/src/main/proto/Plan.proto +++ b/tajo-plan/src/main/proto/Plan.proto @@ -328,25 +328,26 @@ enum EvalType { WINDOW_FUNCTION = 18; AGG_FUNCTION = 19; FUNCTION = 20; + PYTHON_FUNCTION = 21; // String operator or pattern matching predicates - LIKE = 21; - SIMILAR_TO = 22; - REGEX = 23; - CONCATENATE = 24; + LIKE = 22; + SIMILAR_TO = 23; + REGEX = 24; + CONCATENATE = 25; // Other predicates - BETWEEN = 25; - CASE = 26; - IF_THEN = 27; - IN = 28; + BETWEEN = 26; + CASE = 27; + IF_THEN = 28; + IN = 29; // Value or Reference - SIGNED = 29; - CAST = 30; - ROW_CONSTANT = 31; - FIELD = 32; - CONST = 33; + SIGNED = 30; + CAST = 31; + ROW_CONSTANT = 32; + FIELD = 33; + CONST = 34; } message EvalNodeTree { @@ -474,6 +475,7 @@ message Datum { optional string text = 7; optional bytes blob = 8; optional Interval interval = 12; + optional Datum actual = 13; // for ANY type datum } message Interval { From 88a2fd2da98b78f52d7b2288a68a2e5ac67ad757 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 26 Mar 2015 22:32:24 +0900 Subject: [PATCH 08/55] TAJO-1344 --- .../src/main/java/org/apache/tajo/json/DatumAdapter.java | 5 ++--- .../org/apache/tajo/engine/function/TestMathFunctions.java | 6 +++--- .../java/org/apache/tajo/engine/query/TestSelectQuery.java | 2 +- .../src/main/java/org/apache/tajo/plan/expr/EvalNode.java | 2 +- 4 files changed, 7 insertions(+), 8 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java b/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java index 4a2533abf8..b9d8aef0c9 100644 --- a/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java +++ b/tajo-common/src/main/java/org/apache/tajo/json/DatumAdapter.java @@ -43,8 +43,7 @@ public Datum deserialize(JsonElement json, Type typeOfT, String[] values = CommonGsonHelper.getOrDie(jsonObject, "value").getAsString().split(","); return new IntervalDatum(Integer.parseInt(values[0]), Long.parseLong(values[1])); case ANY: - return new AnyDatum(context.deserialize(CommonGsonHelper.getOrDie(jsonObject, "actual"), - DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName)))); + return new AnyDatum(deserialize(CommonGsonHelper.getOrDie(jsonObject, "actual"), typeOfT, context)); default: return context.deserialize(CommonGsonHelper.getOrDie(jsonObject, "body"), DatumFactory.getDatumClass(TajoDataTypes.Type.valueOf(typeName))); @@ -70,7 +69,7 @@ public JsonElement serialize(Datum src, Type typeOfSrc, JsonSerializationContext jsonObj.addProperty("value", interval.getMonths() + "," + interval.getMilliSeconds()); break; case ANY: - jsonObj.add("actual", context.serialize(((AnyDatum) src).getActual())); + jsonObj.add("actual", serialize(((AnyDatum) src).getActual(), typeOfSrc, context)); break; default: jsonObj.add("body", context.serialize(src)); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java index 6a3cec708c..8ac7041907 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java @@ -472,9 +472,9 @@ public void testRoundWithSpecifiedPrecision() throws IOException { @Test public void test() throws IOException { // testSimpleEval("select pow(1,2)", new String[]{"1.0"}); -// testSimpleEval("select sum_py(1,2)", new String[]{"3"}); -// testSimpleEval("select return_one()", new String[]{"100"}); -// testSimpleEval("select helloworld()", new String[]{"Hello, World"}); + testSimpleEval("select sum_py(1,2)", new String[]{"3"}); + testSimpleEval("select return_one()", new String[]{"100"}); + testSimpleEval("select helloworld()", new String[]{"Hello, World"}); testSimpleEval("select concat_py('1')", new String[]{"11"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 06203cb723..7c5caeae16 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -711,7 +711,7 @@ public void testMultiBytesDelimiter4() throws Exception { @Test public void test() throws Exception { - ResultSet res = executeString("select concat_py(n_name) from nation where n_nationkey = 1"); + ResultSet res = executeString("select sum_py(n_nationkey, n_regionkey), concat_py(n_name) from nation where n_nationkey < 5"); System.out.println(resultSetToString(res)); } } \ No newline at end of file diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java index dcb7285a33..f81bf461fb 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java @@ -57,7 +57,7 @@ public EvalType getType() { @Override public String toJson() { - return PlanGsonHelper.toJson(this, EvalNode.class); + return PlanGsonHelper.getPrettyInstance().toJson(this, EvalNode.class); } public abstract T eval(Schema schema, Tuple tuple); From b615a696ab42c75fd4832592772c7219b6a4dccc Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 26 Mar 2015 23:22:10 +0900 Subject: [PATCH 09/55] TAJO-1344 --- .../org/apache/tajo/engine/function/TestMathFunctions.java | 1 - .../apache/tajo/plan/function/python/JythonScriptEngine.java | 3 ++- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java index 8ac7041907..77011110a6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java @@ -471,7 +471,6 @@ public void testRoundWithSpecifiedPrecision() throws IOException { @Test public void test() throws IOException { -// testSimpleEval("select pow(1,2)", new String[]{"1.0"}); testSimpleEval("select sum_py(1,2)", new String[]{"3"}); testSimpleEval("select return_one()", new String[]{"100"}); testSimpleEval("select helloworld()", new String[]{"Hello, World"}); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 4cf022dcd7..1ca59ffca1 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -106,7 +106,8 @@ private static class Interpreter { if (jyhome != null) { Py.getSystemState().path.append(new PyString(jyhome + File.separator + "Lib")); } - LOG.info(ClassLoader.getSystemResource("python")); + URL pythonPath = ClassLoader.getSystemResource("/usr/bin/python"); + LOG.info(pythonPath == null ? "null" : pythonPath); Py.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").toString())); } catch (Exception e) { LOG.warn("issue with jython cache dir", e); From 99ec66a0747413fb2e43a3b15dc142fb4a943b41 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 26 Mar 2015 23:44:11 +0900 Subject: [PATCH 10/55] TAJO-1344 --- .../tajo/plan/function/python/JythonScriptEngine.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 1ca59ffca1..a86af64835 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -106,9 +106,9 @@ private static class Interpreter { if (jyhome != null) { Py.getSystemState().path.append(new PyString(jyhome + File.separator + "Lib")); } - URL pythonPath = ClassLoader.getSystemResource("/usr/bin/python"); + URL pythonPath = ClassLoader.getSystemResource("python"); LOG.info(pythonPath == null ? "null" : pythonPath); - Py.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").toString())); +// Py.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").toString())); } catch (Exception e) { LOG.warn("issue with jython cache dir", e); } @@ -209,8 +209,8 @@ static void execfile(OptionalFunctionContext context, InputStream script, String before.keySet().removeAll(includePyModules); } - LOG.info(ClassLoader.getSystemResource("python").getPath()); - interpreter.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").getPath())); +// LOG.info(ClassLoader.getSystemResource("python").getPath()); +// interpreter.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").getPath())); // exec the code, arbitrary imports are processed interpreter.execfile(script, path); From 3457b7830e12f8a8273b9d76f35198bacbee8f90 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Mar 2015 11:09:07 +0900 Subject: [PATCH 11/55] Remove OptionalFunctionContext --- .../tajo/engine/function/FunctionLoader.java | 4 +- .../org/apache/tajo/master/TajoMaster.java | 11 ++--- .../org/apache/tajo/worker/TajoWorker.java | 5 +- .../apache/tajo/engine/eval/ExprTestBase.java | 3 +- .../python/TestJythonScriptEngine.java | 4 +- .../function/OptionalFunctionContext.java | 49 ------------------- .../function/python/JythonScriptEngine.java | 39 +++++++-------- .../plan/function/python/JythonUtils.java | 21 -------- .../function/python/TajoScriptEngine.java | 11 ----- 9 files changed, 28 insertions(+), 119 deletions(-) delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/OptionalFunctionContext.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index 1fe74e32ab..0477965427 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -37,7 +37,6 @@ import org.apache.tajo.engine.function.annotation.ParamOptionTypes; import org.apache.tajo.engine.function.annotation.ParamTypes; import org.apache.tajo.function.*; -import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.plan.function.python.JythonScriptEngine; import org.apache.tajo.util.ClassUtil; import org.apache.tajo.util.TUtil; @@ -79,7 +78,6 @@ public static Map load() { } public static Map loadOptionalFunctions(TajoConf conf, - OptionalFunctionContext context, Map functionMap) throws IOException { @@ -97,7 +95,7 @@ public static Map loadOptionalFunctions(TajoCon filePaths.add(codePath); } for (Path filePath : filePaths) { - for (FunctionDesc f : JythonScriptEngine.registerFunctions(context, filePath.toString(), + for (FunctionDesc f : JythonScriptEngine.registerFunctions(filePath.toString(), FunctionLoader.PYTHON_FUNCTION_NAMESPACE)) { functionMap.put(f.getSignature(), f); LOG.info(f); diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 528644e816..3d1f4a3535 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -43,7 +43,6 @@ import org.apache.tajo.master.rm.WorkerResourceManager; import org.apache.tajo.metrics.CatalogMetricsGaugeSet; import org.apache.tajo.metrics.WorkerResourceMetricsGaugeSet; -import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.rpc.RpcChannelFactory; import org.apache.tajo.rule.EvaluationContext; import org.apache.tajo.rule.EvaluationFailedException; @@ -137,7 +136,7 @@ public class TajoMaster extends CompositeService { private HistoryReader historyReader; // context for non-builtin functions - private OptionalFunctionContext functionContext = new OptionalFunctionContext(); +// private OptionalFunctionContext functionContext = new OptionalFunctionContext(); public TajoMaster() throws Exception { super(TajoMaster.class.getName()); @@ -209,7 +208,7 @@ public void serviceInit(Configuration _conf) throws Exception { private Collection loadFunctions() throws IOException { Map functionMap = FunctionLoader.load(); - return FunctionLoader.loadOptionalFunctions(systemConf, functionContext, functionMap).values(); + return FunctionLoader.loadOptionalFunctions(systemConf, functionMap).values(); } private void initSystemMetrics() { @@ -485,9 +484,9 @@ public HistoryReader getHistoryReader() { return historyReader; } - public OptionalFunctionContext getFunctionContext() { - return functionContext; - } +// public OptionalFunctionContext getFunctionContext() { +// return functionContext; +// } } String getThreadTaskName(long id, String name) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index c393b90a48..1c157ea7de 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -37,7 +37,6 @@ import org.apache.tajo.conf.TajoConf; import org.apache.tajo.engine.function.FunctionLoader; import org.apache.tajo.function.FunctionSignature; -import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.service.ServiceTracker; import org.apache.tajo.service.ServiceTrackerFactory; import org.apache.tajo.service.TajoMasterInfo; @@ -147,7 +146,7 @@ public class TajoWorker extends CompositeService { private HistoryReader historyReader; // context for non-builtin functions - private OptionalFunctionContext functionContext = new OptionalFunctionContext(); +// private OptionalFunctionContext functionContext = new OptionalFunctionContext(); public TajoWorker() throws Exception { super(TajoWorker.class.getName()); @@ -284,7 +283,7 @@ public void serviceInit(Configuration conf) throws Exception { historyReader = new HistoryReader(workerContext.getWorkerName(), this.systemConf); - FunctionLoader.loadOptionalFunctions(systemConf, functionContext, new HashMap()); + FunctionLoader.loadOptionalFunctions(systemConf, new HashMap()); diagnoseTajoWorker(); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 9c0be89351..e0208e0e1d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -39,7 +39,6 @@ import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.plan.*; import org.apache.tajo.plan.expr.EvalNode; -import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.plan.serder.EvalNodeDeserializer; import org.apache.tajo.plan.serder.EvalNodeSerializer; import org.apache.tajo.engine.query.QueryContext; @@ -95,7 +94,7 @@ public static void setUp() throws Exception { cat.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); cat.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); Map map = FunctionLoader.load(); - map = FunctionLoader.loadOptionalFunctions(conf, new OptionalFunctionContext(), map); + map = FunctionLoader.loadOptionalFunctions(conf, map); for (FunctionDesc funcDesc : map.values()) { cat.createFunction(funcDesc); } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java index c8ad916195..e4770f59c1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java @@ -21,7 +21,6 @@ import junit.framework.TestCase; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.plan.function.python.JythonScriptEngine; import org.apache.tajo.util.FileUtil; import org.python.core.PyFunction; @@ -40,7 +39,6 @@ public void testGetFunction() throws Exception { } public void testRegisterFunction() throws Exception { - OptionalFunctionContext context = new OptionalFunctionContext(); - JythonScriptEngine.registerFunctions(context, "python/test1.py", "test"); + JythonScriptEngine.registerFunctions("python/test1.py", "test"); } } \ No newline at end of file diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/OptionalFunctionContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/OptionalFunctionContext.java deleted file mode 100644 index e812abf7cc..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/OptionalFunctionContext.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function; - -import java.io.File; -import java.util.LinkedHashMap; -import java.util.Map; - -public class OptionalFunctionContext { - - private Map aliasedScriptFiles = new LinkedHashMap(); - - /** - * this method adds script files that must be added to the shipped jar - * named differently from their local fs path. - * @param name name in the jar - * @param path path on the local fs - */ - public void addScriptFile(String name, String path) { - if (path != null) { - aliasedScriptFiles.put(name.replaceFirst("^/", "").replaceAll(":", ""), new File(path)); - } - } - - /** - * calls: addScriptFile(path, new File(path)), ensuring that a given path is - * added to the jar at most once. - * @param path - */ - public void addScriptFile(String path) { - addScriptFile(path, path); - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index a86af64835..1dad961767 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -42,7 +42,6 @@ import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.function.FunctionSupplement; import org.apache.tajo.function.PythonInvocationDesc; -import org.apache.tajo.plan.function.OptionalFunctionContext; import org.apache.tajo.util.TUtil; import org.python.core.*; import org.python.modules.zipimport.zipimporter; @@ -89,7 +88,7 @@ private static class Interpreter { } } if (tmp == null) { - tmp = File.createTempFile("pig_jython_", ""); + tmp = File.createTempFile("tajo_jython_", ""); tmp.delete(); if (tmp.mkdirs() == false) { LOG.warn("unable to create a tmp dir for the cache, jython may not work"); @@ -106,9 +105,6 @@ private static class Interpreter { if (jyhome != null) { Py.getSystemState().path.append(new PyString(jyhome + File.separator + "Lib")); } - URL pythonPath = ClassLoader.getSystemResource("python"); - LOG.info(pythonPath == null ? "null" : pythonPath); -// Py.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").toString())); } catch (Exception e) { LOG.warn("issue with jython cache dir", e); } @@ -120,11 +116,11 @@ private static class Interpreter { /** * ensure the decorator functions are defined in the interpreter, and * manage the module import dependencies. + * @param context * @param path location of a file to exec in the interpreter - * @param queryContext if non-null, module import state is tracked * @throws IOException */ - static synchronized void init(OptionalFunctionContext context, String path) throws IOException { + static synchronized void init(boolean init, String path) throws IOException { // Decorators - // "schemaFunction" // "outputSchema" @@ -155,7 +151,7 @@ static synchronized void init(OptionalFunctionContext context, String path) thro throw new IllegalStateException("unable to create a stream for path: " + path); } try { - execfile(context, is, path); + execfile(init, is, path); } finally { is.close(); } @@ -169,10 +165,11 @@ static synchronized void init(OptionalFunctionContext context, String path) thro * @param queryContext * @throws Exception */ - static void execfile(OptionalFunctionContext context, InputStream script, String path) throws RuntimeException { + static void execfile(boolean init, InputStream script, String path) throws RuntimeException { try { - if( context != null ) { +// if( context != null ) { + if (init) { String [] argv; try { // argv = (String[])ObjectSerializer.deserialize( @@ -194,7 +191,8 @@ static void execfile(OptionalFunctionContext context, InputStream script, String } // determine the current module state - Map before = context != null ? getModuleState() : null; +// Map before = context != null ? getModuleState() : null; + Map before = init ? getModuleState() : null; if (before != null) { // os.py, stax.py and posixpath.py are part of the initial state // if Lib directory is present and without including os.py, modules @@ -209,17 +207,16 @@ static void execfile(OptionalFunctionContext context, InputStream script, String before.keySet().removeAll(includePyModules); } -// LOG.info(ClassLoader.getSystemResource("python").getPath()); -// interpreter.getSystemState().path.append(new PyString(ClassLoader.getSystemResource("python").getPath())); - // exec the code, arbitrary imports are processed interpreter.execfile(script, path); // determine the 'post import' module state - Map after = context != null ? getModuleState() : null; +// Map after = context != null ? getModuleState() : null; + Map after = init ? getModuleState() : null; // add the module files to the context - if (after != null && context != null) { +// if (after != null && context != null) { + if (after != null && init) { after.keySet().removeAll(before.keySet()); for (Map.Entry entry : after.entrySet()) { String modulename = entry.getKey(); @@ -230,7 +227,7 @@ static void execfile(OptionalFunctionContext context, InputStream script, String // context.addScriptJar(modulepath); throw new RuntimeException("jar and zip script files are not supported"); } else { - context.addScriptFile(modulename, modulepath); +// context.addScriptFile(modulename, modulepath); } } } @@ -347,7 +344,7 @@ private static File resolvePyModulePath(String path, Object loader) { */ public static PyFunction getFunction(String path, String functionName) throws IOException { Interpreter.setMain(false); - Interpreter.init(null, path); + Interpreter.init(false, path); return (PyFunction) Interpreter.interpreter.get(functionName); } @@ -428,10 +425,10 @@ private static boolean delete(final File file) { // @Override // public void registerFunctions(String path, String namespace, QueryContext context) - public static Set registerFunctions(OptionalFunctionContext context, String path, String namespace) + public static Set registerFunctions(String path, String namespace) throws IOException { Interpreter.setMain(false); - Interpreter.init(context, path); + Interpreter.init(true, path); // context.addScriptJar(getJarPath(PythonInterpreter.class)); PythonInterpreter pi = Interpreter.interpreter; @SuppressWarnings("unchecked") @@ -479,7 +476,7 @@ public static Set registerFunctions(OptionalFunctionContext contex } } - context.addScriptFile(path); +// context.addScriptFile(path); Interpreter.setMain(true); return functionDescs; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java index 9298be43f5..347849ea93 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java @@ -29,27 +29,6 @@ public class JythonUtils { -// /** -// * @param schemaString a String representation of the Schema without -// * any enclosing curly-braces.Not for use with -// * Schema#toString -// * @return Schema instance -// * @throws ParserException -// */ -// public static Schema getSchemaFromString(String schemaString) { -// LogicalSchema schema = parseSchema(schemaString); -// Schema result = org.apache.pig.newplan.logical.Util.translateSchema(schema); -// Schema.setSchemaDefaultType(result, DataType.BYTEARRAY); -// return result; -// } -// -// public static LogicalSchema parseSchema(String schemaString) { -// QueryParserDriver queryParser = new QueryParserDriver( new PigContext(), -// "util", new HashMap() ) ; -// LogicalSchema schema = queryParser.parseSchema(schemaString); -// return schema; -// } - public static PyObject datumToPyObject(Datum v) { Preconditions.checkArgument(v.type() == TajoDataTypes.Type.ANY); Datum actual = ((AnyDatum) v).getActual(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index c52e2a7aa5..2d52ab2655 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -93,17 +93,6 @@ private static InputStream getResourceUsingClassLoader(String fullFilename, Clas */ protected abstract Map getParamsFromVariables() throws IOException; -// /** -// * Registers scripting language functions as Pig functions with given namespace -// * -// * @param path path of the script -// * @param namespace namespace for the functions -// * @param context context to register functions to tajo in the given namespace -// * @throws IOException -// */ -// public abstract void registerFunctions(String path, String namespace, -// QueryContext context) throws IOException; - /** * Figures out the jar location from the class * @param clazz class in the jar file From 4741be6bd7faf96ed1cc8d22e430388db84c3e4e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Mar 2015 12:20:09 +0900 Subject: [PATCH 12/55] TAJO-1344 --- .../org/apache/tajo/master/TajoMaster.java | 7 ------- .../org/apache/tajo/worker/TajoWorker.java | 18 ------------------ 2 files changed, 25 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 3d1f4a3535..397d552bd7 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -135,9 +135,6 @@ public class TajoMaster extends CompositeService { private HistoryReader historyReader; - // context for non-builtin functions -// private OptionalFunctionContext functionContext = new OptionalFunctionContext(); - public TajoMaster() throws Exception { super(TajoMaster.class.getName()); } @@ -483,10 +480,6 @@ public HistoryWriter getHistoryWriter() { public HistoryReader getHistoryReader() { return historyReader; } - -// public OptionalFunctionContext getFunctionContext() { -// return functionContext; -// } } String getThreadTaskName(long id, String name) { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 1c157ea7de..9e8980aeb4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -145,9 +145,6 @@ public class TajoWorker extends CompositeService { private HistoryReader historyReader; - // context for non-builtin functions -// private OptionalFunctionContext functionContext = new OptionalFunctionContext(); - public TajoWorker() throws Exception { super(TajoWorker.class.getName()); } @@ -288,21 +285,6 @@ public void serviceInit(Configuration conf) throws Exception { diagnoseTajoWorker(); } -// private void initOptionalFunctions() throws IOException { -// String[] codePaths = systemConf.getStrings(TajoConf.ConfVars.PYTHON_CODE_DIR.varname); -// for (String codePath : codePaths) { -// Set functions = JythonScriptEngine.registerFunctions(functionContext, codePath, -// FunctionLoader.PYTHON_FUNCTION_NAMESPACE); -// for (FunctionDesc desc : functions) { -// LOG.info("FunctionLoader: " + desc); -// } -//// for (FunctionDesc f : JythonScriptEngine.registerFunctions(functionContext, codePath, -//// FunctionLoader.PYTHON_FUNCTION_NAMESPACE)) { -//// map.put(f.getSignature(), f); -//// } -// } -// } - private void initWorkerMetrics() { workerSystemMetrics = new TajoSystemMetrics(systemConf, "worker", workerContext.getWorkerName()); workerSystemMetrics.start(); From 4a5570597b85f71af630d3c318de87aa03737ae7 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Mar 2015 18:01:40 +0900 Subject: [PATCH 13/55] TAJO-1344 --- .../tajo/function/PythonInvocationDesc.java | 5 +- .../org/apache/tajo/TajoTestingCluster.java | 2 +- .../engine/function/TestMathFunctions.java | 8 - .../engine/function/TestPythonFunctions.java | 36 +++ .../python/TestJythonScriptEngine.java | 4 +- .../tajo/engine/query/TestSelectQuery.java | 7 +- .../python/{test1.py => test_funcs.py} | 0 .../TestSelectQuery/testSelectPythonFuncs.sql | 2 + .../testSelectPythonFuncs.result | 7 + .../plan/expr/GeneralPythonFunctionEval.java | 4 +- .../function/python/JythonScriptEngine.java | 256 +++--------------- .../plan/function/python/JythonUtils.java | 86 ++++-- .../function/python/TajoScriptEngine.java | 36 +-- 13 files changed, 177 insertions(+), 276 deletions(-) create mode 100644 tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java rename tajo-core/src/test/resources/python/{test1.py => test_funcs.py} (100%) create mode 100644 tajo-core/src/test/resources/queries/TestSelectQuery/testSelectPythonFuncs.sql create mode 100644 tajo-core/src/test/resources/results/TestSelectQuery/testSelectPythonFuncs.result diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java index 2d995f2de8..8217cd7e15 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java @@ -57,7 +57,8 @@ public String getPath() { @Override public PythonInvocationDescProto getProto() { - return PythonInvocationDescProto.newBuilder() - .setFuncName(funcName).setFilePath(filePath).build(); + PythonInvocationDescProto.Builder builder = PythonInvocationDescProto.newBuilder(); + builder.setFuncName(funcName).setFilePath(filePath); + return builder.build(); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 25c2dc63dd..1bf04b250b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -165,7 +165,7 @@ void initPropertiesAndConfigs() { // Memory cache termination conf.setIntVar(ConfVars.WORKER_HISTORY_EXPIRE_PERIOD, 1); - conf.setStrings(ConfVars.PYTHON_CODE_DIR.varname, "python/test1.py"); + conf.setStrings(ConfVars.PYTHON_CODE_DIR.varname, "python/test_funcs.py"); this.standbyWorkerMode = conf.getVar(ConfVars.RESOURCE_MANAGER_CLASS) .indexOf(TajoWorkerResourceManager.class.getName()) >= 0; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java index 77011110a6..78509f77e1 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestMathFunctions.java @@ -468,12 +468,4 @@ public void testRoundWithSpecifiedPrecision() throws IOException { testEval(schema, "table1", ",", "select round(col1, col2) from table1", new String[]{""}); } - - @Test - public void test() throws IOException { - testSimpleEval("select sum_py(1,2)", new String[]{"3"}); - testSimpleEval("select return_one()", new String[]{"100"}); - testSimpleEval("select helloworld()", new String[]{"Hello, World"}); - testSimpleEval("select concat_py('1')", new String[]{"11"}); - } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java new file mode 100644 index 0000000000..48039229e9 --- /dev/null +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.engine.function; + +import org.apache.tajo.engine.eval.ExprTestBase; +import org.junit.Test; + +import java.io.IOException; + +public class TestPythonFunctions extends ExprTestBase { + + @Test + public void test() throws IOException { + testSimpleEval("select sum_py(1,2)", new String[]{"3"}); + testSimpleEval("select return_one()", new String[]{"100"}); + testSimpleEval("select helloworld()", new String[]{"Hello, World"}); + testSimpleEval("select concat_py('1')", new String[]{"11"}); + testSimpleEval("select square_py(32)", new String[]{"1024"}); + } +} diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java index e4770f59c1..ca9d4b7d49 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java @@ -31,7 +31,7 @@ public class TestJythonScriptEngine extends TestCase { static final Log LOG = LogFactory.getLog(TestJythonScriptEngine.class); public void testGetFunction() throws Exception { - URL url = FileUtil.getResourcePath("python/test1.py"); + URL url = FileUtil.getResourcePath("python/test_funcs.py"); LOG.info("File path: " + url); PyFunction function = JythonScriptEngine.getFunction(url.getPath(), "return_one"); LOG.info(function.getType()); @@ -39,6 +39,6 @@ public void testGetFunction() throws Exception { } public void testRegisterFunction() throws Exception { - JythonScriptEngine.registerFunctions("python/test1.py", "test"); + JythonScriptEngine.registerFunctions("python/test_funcs.py", "test"); } } \ No newline at end of file diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 7c5caeae16..03916939e4 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -710,8 +710,9 @@ public void testMultiBytesDelimiter4() throws Exception { } @Test - public void test() throws Exception { - ResultSet res = executeString("select sum_py(n_nationkey, n_regionkey), concat_py(n_name) from nation where n_nationkey < 5"); - System.out.println(resultSetToString(res)); + public void testSelectPythonFuncs() throws Exception { + ResultSet res = executeQuery(); + assertResultSet(res); + cleanupQuery(res); } } \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/test1.py b/tajo-core/src/test/resources/python/test_funcs.py similarity index 100% rename from tajo-core/src/test/resources/python/test1.py rename to tajo-core/src/test/resources/python/test_funcs.py diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testSelectPythonFuncs.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testSelectPythonFuncs.sql new file mode 100644 index 0000000000..bcb9806ed3 --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testSelectPythonFuncs.sql @@ -0,0 +1,2 @@ +select helloworld(), sum_py(n_nationkey, n_regionkey) as sum, concat_py(n_name) as concat +from nation where n_nationkey < 5 \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testSelectPythonFuncs.result b/tajo-core/src/test/resources/results/TestSelectQuery/testSelectPythonFuncs.result new file mode 100644 index 0000000000..877e9fc3c1 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestSelectQuery/testSelectPythonFuncs.result @@ -0,0 +1,7 @@ +?helloworld,sum,concat +------------------------------- +Hello, World,0,ALGERIAALGERIA +Hello, World,2,ARGENTINAARGENTINA +Hello, World,3,BRAZILBRAZIL +Hello, World,4,CANADACANADA +Hello, World,8,EGYPTEGYPT \ No newline at end of file diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index 0d22a8975d..b50d37e61e 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -50,9 +50,9 @@ public Datum eval(Schema schema, Tuple tuple) { params.put(i, argEvals[i].eval(schema, tuple)); } } - PythonInvocationDesc desc = funcDesc.getInvocation().getPython(); + PythonInvocationDesc invokeDesc = funcDesc.getInvocation().getPython(); try { - PyFunction function = JythonScriptEngine.getFunction(desc.getPath(), desc.getName()); + PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); TajoDataTypes.DataType[] paramTypes = funcDesc.getSignature().getParamTypes(); PyObject result; if (paramTypes.length == 0) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 1dad961767..059c4e7ce0 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -18,37 +18,29 @@ package org.apache.tajo.plan.function.python; -import java.io.BufferedReader; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.InputStreamReader; -import java.net.URL; import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.regex.Pattern; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.conf.TajoConf.ConfVars; +import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.function.FunctionInvocation; import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.function.FunctionSupplement; import org.apache.tajo.function.PythonInvocationDesc; import org.apache.tajo.util.TUtil; import org.python.core.*; -import org.python.modules.zipimport.zipimporter; import org.python.util.PythonInterpreter; -import javax.script.ScriptEngine; - /** * Implementation of the script engine for Jython */ @@ -82,7 +74,7 @@ private static class Interpreter { String cdir = System.getProperty(PySystemState.PYTHON_CACHEDIR); if (cdir != null) { tmp = new File(cdir); - if (tmp.canWrite() == false) { + if (!tmp.canWrite()) { LOG.error("CACHEDIR: not writable"); throw new RuntimeException("python.cachedir not writable: " + cdir); } @@ -90,7 +82,7 @@ private static class Interpreter { if (tmp == null) { tmp = File.createTempFile("tajo_jython_", ""); tmp.delete(); - if (tmp.mkdirs() == false) { + if (!tmp.mkdirs()) { LOG.warn("unable to create a tmp dir for the cache, jython may not work"); } else { LOG.info("created tmp python.cachedir=" + tmp); @@ -114,13 +106,13 @@ private static class Interpreter { } /** - * ensure the decorator functions are defined in the interpreter, and + * Ensure the decorator functions are defined in the interpreter, and * manage the module import dependencies. - * @param context - * @param path location of a file to exec in the interpreter + * @param initPhase True if the script is not registered. Otherwise false. + * @param path location of a script file to exec in the interpreter * @throws IOException */ - static synchronized void init(boolean init, String path) throws IOException { + static synchronized void init(boolean initPhase, String path) throws IOException { // Decorators - // "schemaFunction" // "outputSchema" @@ -134,24 +126,25 @@ static synchronized void init(boolean init, String path) throws IOException { + " return func\n" + " return decorator\n\n"); - interpreter.exec("def outputSchemaFunction(schema_def):\n" - + " def decorator(func):\n" - + " func.outputSchemaFunction = schema_def\n" - + " return func\n" - + " return decorator\n"); - - interpreter.exec("def schemaFunction(schema_def):\n" - + " def decorator(func):\n" - + " func.schemaFunction = schema_def\n" - + " return func\n" - + " return decorator\n\n"); + // TODO: Currently, we don't support the customized output schema feature. +// interpreter.exec("def outputSchemaFunction(schema_def):\n" +// + " def decorator(func):\n" +// + " func.outputSchemaFunction = schema_def\n" +// + " return func\n" +// + " return decorator\n"); +// +// interpreter.exec("def schemaFunction(schema_def):\n" +// + " def decorator(func):\n" +// + " func.schemaFunction = schema_def\n" +// + " return func\n" +// + " return decorator\n\n"); InputStream is = getScriptAsStream(path); if (is == null) { throw new IllegalStateException("unable to create a stream for path: " + path); } try { - execfile(init, is, path); + execfile(initPhase, is, path); } finally { is.close(); } @@ -160,77 +153,15 @@ static synchronized void init(boolean init, String path) throws IOException { /** * does not call script.close() - * @param script - * @param path - * @param queryContext + * @param initPhase True if the script is not registered. Otherwise false. + * @param script Input stream to the script file + * @param path Path to the script file * @throws Exception */ - static void execfile(boolean init, InputStream script, String path) throws RuntimeException { + static void execfile(boolean initPhase, InputStream script, String path) throws RuntimeException { try { - -// if( context != null ) { - if (init) { - String [] argv; - try { -// argv = (String[])ObjectSerializer.deserialize( -// queryContext.get(ConfVars.PYTHON_CMD_ARGS_REMAINDERS)); - argv = new String [] {}; - } catch (Exception e) { - throw new RuntimeException("Cannot deserialize command line arguments", e); - } - PySystemState state = Py.getSystemState(); - state.argv.clear(); - if( argv != null ) { - for (String str : argv ) { - state.argv.append(new PyString(str)); - } - } else { - LOG.warn(ConfVars.PYTHON_CMD_ARGS_REMAINDERS.name() - + " is empty. This is not expected unless on testing." ); - } - } - - // determine the current module state -// Map before = context != null ? getModuleState() : null; - Map before = init ? getModuleState() : null; - if (before != null) { - // os.py, stax.py and posixpath.py are part of the initial state - // if Lib directory is present and without including os.py, modules - // which import os fail - Set includePyModules = new HashSet(); - for (String key : before.keySet()) { - // $py.class is created if Lib folder is writable - if (key.endsWith(".py") || key.endsWith("$py.class")) { - includePyModules.add(key); - } - } - before.keySet().removeAll(includePyModules); - } - // exec the code, arbitrary imports are processed interpreter.execfile(script, path); - - // determine the 'post import' module state -// Map after = context != null ? getModuleState() : null; - Map after = init ? getModuleState() : null; - - // add the module files to the context -// if (after != null && context != null) { - if (after != null && init) { - after.keySet().removeAll(before.keySet()); - for (Map.Entry entry : after.entrySet()) { - String modulename = entry.getKey(); - String modulepath = entry.getValue(); - if (modulepath.equals(JVM_JAR)) { - continue; - } else if (modulepath.endsWith(".jar") || modulepath.endsWith(".zip")) { -// context.addScriptJar(modulepath); - throw new RuntimeException("jar and zip script files are not supported"); - } else { -// context.addScriptFile(modulename, modulepath); - } - } - } } catch (PyException e) { if (e.match(Py.SystemExit)) { PyObject value = e.value; @@ -243,15 +174,10 @@ static void execfile(boolean init, InputStream script, String path) throws Runti } } String message = "Python Error. " + e; - //throw new ExecException(message, 1121, e); throw new RuntimeException(message, e); } } - static String get(String name) { - return interpreter.get(name).toString(); - } - static void setMain(boolean isMain) { if (isMain) { interpreter.set("__name__", "__main__"); @@ -259,80 +185,6 @@ static void setMain(boolean isMain) { interpreter.set("__name__", "__lib__"); } } - - /** - * get the state of modules currently loaded - * @return a map of module name to module file (absolute path) - */ - private static Map getModuleState() { - // determine the current module state - Map files = new HashMap(); - PyStringMap modules = (PyStringMap) Py.getSystemState().modules; - for (PyObject kvp : modules.iteritems().asIterable()) { - PyTuple tuple = (PyTuple) kvp; - String name = tuple.get(0).toString(); - Object value = tuple.get(1); - // inspect the module to determine file location and status - try { - Object fileEntry = null; - Object loader = null; - if (value instanceof PyJavaPackage ) { - fileEntry = ((PyJavaPackage) value).__file__; - } else if (value instanceof PyObject) { - // resolved through the filesystem (or built-in) - PyObject dict = ((PyObject) value).getDict(); - if (dict != null) { - fileEntry = dict.__finditem__("__file__"); - loader = dict.__finditem__("__loader__"); - } // else built-in - } // else some system module? - - if (fileEntry != null) { - File file = resolvePyModulePath(fileEntry.toString(), loader); - if (file.exists()) { - String apath = file.getAbsolutePath(); - if (apath.endsWith(".jar") || apath.endsWith(".zip")) { - // jar files are simple added to the pigContext - files.put(apath, apath); - } else { - // determine the relative path that the file should have in the jar - int pos = apath.lastIndexOf(File.separatorChar + name.replace('.', File.separatorChar)); - if (pos > 0) { - files.put(apath.substring(pos + 1), apath); - } else { - files.put(apath, apath); - } - } - } else { - LOG.warn("module file does not exist: " + name + ", " + file); - } - } // else built-in - } catch (Exception e) { - LOG.warn("exception while retrieving module state: " + value, e); - } - } - return files; - } - } - - private static File resolvePyModulePath(String path, Object loader) { - File file = new File(path); - if (!file.exists() && loader != null) { - if(path.startsWith(ClasspathPyImporter.PYCLASSPATH_PREFIX) && loader instanceof ClasspathPyImporter) { - path = path.replaceFirst(ClasspathPyImporter.PYCLASSPATH_PREFIX, ""); - URL resource = ScriptEngine.class.getResource(path); - if (resource == null) { - resource = ScriptEngine.class.getResource(File.separator + path); - } - if (resource != null) { - return new File(resource.getFile()); - } - } else if (loader instanceof zipimporter) { - zipimporter importer = (zipimporter) loader; - return new File(importer.archive); - } //JavaImporter?? - } - return file; } /** @@ -370,31 +222,6 @@ protected Map getParamsFromVariables() throws IOException { return vars; } - private static final Pattern p = Pattern.compile("^\\s*def\\s+(\\w+)\\s*.+"); - private static final Pattern p1 = Pattern.compile("^\\s*if\\s+__name__\\s+==\\s+[\"']__main__[\"']\\s*:\\s*$"); - - private static boolean hasFunction(InputStream is) throws IOException { - boolean hasFunction = false; - boolean hasMain = false; - InputStreamReader in = new InputStreamReader(is); - BufferedReader br = new BufferedReader(in); - String line = br.readLine(); - while (line != null) { - if (p.matcher(line).matches()) { - hasFunction = true; - } else if (p1.matcher(line).matches()) { - hasMain = true; - } - line = br.readLine(); - } - if (hasFunction && !hasMain) { - String msg = "Embedded script cannot mix UDFs with top level code. " - + "Please use if __name__ == '__main__': construct"; - throw new IOException(msg); - } - return hasFunction; - } - /** * File.deleteOnExit(File) does not work for a non-empty directory. This * Thread is used to clean up the python.cachedir (if it was a tmp dir @@ -423,13 +250,10 @@ private static boolean delete(final File file) { } } - // @Override -// public void registerFunctions(String path, String namespace, QueryContext context) public static Set registerFunctions(String path, String namespace) throws IOException { Interpreter.setMain(false); Interpreter.init(true, path); -// context.addScriptJar(getJarPath(PythonInterpreter.class)); PythonInterpreter pi = Interpreter.interpreter; @SuppressWarnings("unchecked") List locals = ((PyStringMap) pi.getLocals()).items(); @@ -445,20 +269,17 @@ public static Set registerFunctions(String path, String namespace) && (value instanceof PyFunction) && (((PyFunction)value).__findattr__("schemaFunction".intern())== null)) { PyFunction pyFunction = (PyFunction) value; - PyObject obj = pyFunction.__findattr__("outputSchema".intern()); + + // Find the pre-defined output schema TajoDataTypes.Type returnType; - if(obj != null) { -// Utils.getSchemaFromString(obj.toString()); - LOG.info("outputSchema: " + obj.toString()); - String[] types = obj.toString().split(","); - if (types.length > 1) { - throw new IOException("Multiple return type is not supported"); - } - returnType = TajoDataTypes.Type.valueOf(types[0].trim().toUpperCase()); + PyObject obj = pyFunction.__findattr__("outputSchema".intern()); + if (obj != null) { + returnType = pyObjectToType(obj); } else { // the default return type is the byte array returnType = TajoDataTypes.Type.BLOB; } + int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; LOG.info("co_argcount: " + paramNum); TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; @@ -469,16 +290,29 @@ public static Set registerFunctions(String path, String namespace) FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, key, TajoDataTypes.DataType.newBuilder().setType(returnType).build(), paramTypes); FunctionInvocation invocation = new FunctionInvocation(); - invocation.setPython(new PythonInvocationDesc(key, path)); + PythonInvocationDesc invocationDesc = new PythonInvocationDesc(key, path); + invocation.setPython(invocationDesc); FunctionSupplement supplement = new FunctionSupplement(); functionDescs.add(new FunctionDesc(signature, invocation, supplement)); LOG.info("Register scripting UDF: " + namespace + key); } } -// context.addScriptFile(path); Interpreter.setMain(true); return functionDescs; } + + private static TajoDataTypes.Type pyObjectToType(PyObject obj) { + return TajoDataTypes.Type.valueOf(pyObjectToTypeStringCand(obj).toUpperCase()); + } + + private static String pyObjectToTypeStringCand(PyObject obj) { + LOG.info("outputSchema: " + obj.toString()); + String[] types = obj.toString().split(","); + if (types.length > 1) { + throw new UnsupportedException("Multiple return type is not supported"); + } + return types[0].trim(); + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java index 347849ea93..a68571ad44 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java @@ -19,6 +19,7 @@ package org.apache.tajo.plan.function.python; import com.google.common.base.Preconditions; +import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.AnyDatum; import org.apache.tajo.datum.Datum; @@ -68,26 +69,6 @@ public static PyObject datumToPyObject(Datum v) { default: throw new UnsupportedException("Unsupported type: " + actual.type()); } - -// if (object instanceof Tuple) { -// return tupleToPyTuple((Tuple) object); -// } else if (object instanceof DataBag) { -// PyList list = new PyList(); -// for (Tuple bagTuple : (DataBag) object) { -// list.add(tupleToPyTuple(bagTuple)); -// } -// return list; -// } else if (object instanceof Map) { -// PyDictionary newMap = new PyDictionary(); -// for (Map.Entry entry : ((Map) object).entrySet()) { -// newMap.put(entry.getKey(), datumToPyObject(entry.getValue())); -// } -// return newMap; -// } else if (object instanceof DataByteArray) { -// return Py.java2py(((DataByteArray) object).get()); -// } else { -// return Py.java2py(object); -// } } public static PyTuple tupleToPyTuple(Tuple tuple) { @@ -130,4 +111,69 @@ public static Datum pyObjectToDatum(PyObject object) { } } } + + public static Datum pyObjectToDatum(PyObject object, TajoDataTypes.Type type) { + return DatumFactory.cast(pyObjectToDatum(object), CatalogUtil.newSimpleDataType(type), null); + } + + public static TajoDataTypes.Type primitiveTypeToDataType(Class clazz) { + if (clazz.getName().equals(Long.class.getName())) { + return TajoDataTypes.Type.INT8; + } else if (clazz.getName().equals(Boolean.class.getName())) { + return TajoDataTypes.Type.BOOLEAN; + } else if (clazz.getName().equals(Integer.class.getName())) { + return TajoDataTypes.Type.INT4; + } else if (clazz.getName().equals(Float.class.getName())) { + // J(P)ython is loosely typed, supports only float type, + // hence we convert everything to double to save precision + return TajoDataTypes.Type.FLOAT4; + } else if (clazz.getName().equals(Double.class.getName())) { + return TajoDataTypes.Type.FLOAT8; + } else if (clazz.getName().equals(String.class.getName())) { + return TajoDataTypes.Type.TEXT; + } else { + if(clazz.getName().equals(byte[].class.getName())) { + return TajoDataTypes.Type.BLOB; + } + else { + throw new UnsupportedException("Not supported data type: " + clazz.getName()); + } + } + } + + public static Object dataTypeToPrimitiveType(TajoDataTypes.Type type) { + switch (type) { + case BOOLEAN: + return Boolean.class; + case UINT1: + case INT1: + case UINT2: + case INT2: + return Short.class; + case UINT4: + case INT4: + return Integer.class; + case UINT8: + case INT8: + return Long.class; + case FLOAT4: + return Float.class; + case FLOAT8: + return Double.class; + case CHAR: + case VARCHAR: + return Character.class; + case TEXT: + case NCHAR: + case NVARCHAR: + return String.class; + case BLOB: + return Byte[].class; + case INET4: + case INET6: + return Byte[].class; + default: + throw new UnsupportedException("Unsupported type: " + type); + } + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 2d52ab2655..626d73b255 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -21,20 +21,23 @@ import org.apache.hadoop.util.Shell; import javax.script.ScriptEngine; -import java.io.*; -import java.net.URL; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.InputStream; +import java.io.IOException; import java.util.Map; public abstract class TajoScriptEngine { + /** - * open a stream load a script locally or in the classpath + * Open a stream load a script locally or in the classpath * @param scriptPath the path of the script * @return a stream (it is the responsibility of the caller to close it) * @throws IllegalStateException if we could not open a stream */ - public static InputStream getScriptAsStream(String scriptPath) { - //protected static InputStream getScriptAsStream(String scriptPath) { - InputStream is = null; + protected static InputStream getScriptAsStream(String scriptPath) { + InputStream is; File file = new File(scriptPath); if (file.exists()) { try { @@ -66,8 +69,6 @@ public static InputStream getScriptAsStream(String scriptPath) { } } - // TODO: discuss if we want to add logic here to load a script from HDFS - if (is == null) { throw new IllegalStateException( "Could not initialize interpreter (from file system or classpath) with " + scriptPath); @@ -92,23 +93,4 @@ private static InputStream getResourceUsingClassLoader(String fullFilename, Clas * @throws java.io.IOException */ protected abstract Map getParamsFromVariables() throws IOException; - - /** - * Figures out the jar location from the class - * @param clazz class in the jar file - * @return the jar file location, null if the class was not loaded from a jar - * @throws FileNotFoundException - */ - public static String getJarPath(Class clazz) - throws FileNotFoundException { - URL resource = clazz.getClassLoader().getResource( - clazz.getCanonicalName().replace(".", "/") + ".class"); - if (resource.getProtocol().equals("jar")) { - return resource.getPath().substring( - resource.getPath().indexOf(':') + 1, - resource.getPath().indexOf('!')); - } - throw new FileNotFoundException("Jar for " + clazz.getName() - + " class is not found"); - } } From 1d7dab11eb73be96e0c979de118e7e4e7321eaac Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Mar 2015 18:14:47 +0900 Subject: [PATCH 14/55] TAJO-1344 --- .../engine/function/TestPythonFunctions.java | 1 - .../python/TestJythonScriptEngine.java | 44 ----------- .../src/test/resources/python/tajo_util.py | 79 ------------------- .../org/apache/tajo/plan/expr/EvalNode.java | 2 +- .../plan/function/python/JythonConstants.java | 26 ++++++ .../function/python/JythonScriptEngine.java | 12 ++- 6 files changed, 32 insertions(+), 132 deletions(-) delete mode 100644 tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java index 48039229e9..1f36832b39 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -31,6 +31,5 @@ public void test() throws IOException { testSimpleEval("select return_one()", new String[]{"100"}); testSimpleEval("select helloworld()", new String[]{"Hello, World"}); testSimpleEval("select concat_py('1')", new String[]{"11"}); - testSimpleEval("select square_py(32)", new String[]{"1024"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java deleted file mode 100644 index ca9d4b7d49..0000000000 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/python/TestJythonScriptEngine.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.engine.function.python; - -import junit.framework.TestCase; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.tajo.plan.function.python.JythonScriptEngine; -import org.apache.tajo.util.FileUtil; -import org.python.core.PyFunction; - -import java.net.URL; - -public class TestJythonScriptEngine extends TestCase { - static final Log LOG = LogFactory.getLog(TestJythonScriptEngine.class); - - public void testGetFunction() throws Exception { - URL url = FileUtil.getResourcePath("python/test_funcs.py"); - LOG.info("File path: " + url); - PyFunction function = JythonScriptEngine.getFunction(url.getPath(), "return_one"); - LOG.info(function.getType()); - LOG.info(function.__call__().toString()); - } - - public void testRegisterFunction() throws Exception { - JythonScriptEngine.registerFunctions("python/test_funcs.py", "test"); - } -} \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/tajo_util.py b/tajo-core/src/test/resources/python/tajo_util.py index 75204d69dd..63ff5b945c 100644 --- a/tajo-core/src/test/resources/python/tajo_util.py +++ b/tajo-core/src/test/resources/python/tajo_util.py @@ -16,88 +16,9 @@ # See the License for the specific language governing permissions and # limitations under the License. -import logging - -class udf_logging(object): - udf_log_level = logging.INFO - - @classmethod - def set_log_level_error(cls): - cls.udf_log_level = logging.ERROR - - @classmethod - def set_log_level_warn(cls): - cls.udf_log_level = logging.WARN - - @classmethod - def set_log_level_info(cls): - cls.udf_log_level = logging.INFO - - @classmethod - def set_log_level_debug(cls): - cls.udf_log_level = logging.DEBUG - def outputSchema(schema_str): def wrap(f): def wrapped_f(*args): return f(*args) return wrapped_f return wrap - -def write_user_exception(filename, stream_err_output, num_lines_offset_trace=0): - import sys - import traceback - import inspect - (t, v, tb) = sys.exc_info() - name = t.__name__ - record_error = False - - if name in ['SyntaxError', 'IndentationError']: - syntax_error_values = v.args - user_line_number = syntax_error_values[1][1] - num_lines_offset_trace - error_message = "%s: %s\n\tFile: %s, line %s column %s\n\t%s" % \ - (name, - syntax_error_values[0], - syntax_error_values[1][0], - user_line_number, - syntax_error_values[1][2], - syntax_error_values[1][3]) - else: - error_message = "%s: %s\n" % (name, v) - user_line_number = None - while 1: - e_file_name = tb.tb_frame.f_code.co_filename - if e_file_name.find(filename) > 0: - record_error = True - if not record_error: - if not tb.tb_next: - break - tb = tb.tb_next - continue - - line_number = tb.tb_lineno - mod = inspect.getmodule(tb) - if mod: - lines, offset = inspect.getsourcelines(mod) - line = lines[line_number - offset - 1] - else: - #Useful to catch exceptions with an invalid module (like syntax - #errors) - lines, offset = inspect.getsourcelines(tb.tb_frame) - if (line_number - 1) >= len(lines): - line = "Unknown Line" - else: - line = lines[line_number - 1] - - user_line_number = line_number - num_lines_offset_trace - func_name = tb.tb_frame.f_code.co_name - error_message += 'File %s, line %s, in %s\n\t%s\n' % \ - (e_file_name, user_line_number, func_name, line) - if not tb.tb_next: - break - tb = tb.tb_next - if name in ['UnicodeEncodeError']: - error_message += "\nTo print a unicode string in your udf use encode('utf-8'). Example: \n\tprint 'Example'.encode('utf-8')" - if user_line_number: - stream_err_output.write("%s\n" % user_line_number) - stream_err_output.write("%s\n" % error_message) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java index f81bf461fb..dcb7285a33 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java @@ -57,7 +57,7 @@ public EvalType getType() { @Override public String toJson() { - return PlanGsonHelper.getPrettyInstance().toJson(this, EvalNode.class); + return PlanGsonHelper.toJson(this, EvalNode.class); } public abstract T eval(Schema schema, Tuple tuple); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java new file mode 100644 index 0000000000..5f980581c2 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.python; + +public class JythonConstants { + public final static String SKIP_TOKEN = "__"; + public final static String OUTPUT_SCHEMA = "outputSchema".intern(); + public final static String OUTPUT_SCHEMA_FUNCTION = "outputSchemaFunction".intern(); + public final static String SCHEMA_FUNCTION = "schemaFunction".intern(); +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 059c4e7ce0..93a9f58cdb 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -263,16 +263,16 @@ public static Set registerFunctions(String path, String namespace) for (PyTuple item : locals) { String key = (String) item.get(0); Object value = item.get(1); - if (!key.startsWith("__") && !key.equals("schemaFunction") - && !key.equals("outputSchema") - && !key.equals("outputSchemaFunction") + if (!key.startsWith(JythonConstants.SKIP_TOKEN) && !key.equals(JythonConstants.SCHEMA_FUNCTION) + && !key.equals(JythonConstants.OUTPUT_SCHEMA) + && !key.equals(JythonConstants.OUTPUT_SCHEMA_FUNCTION) && (value instanceof PyFunction) - && (((PyFunction)value).__findattr__("schemaFunction".intern())== null)) { + && (((PyFunction)value).__findattr__(JythonConstants.SCHEMA_FUNCTION)== null)) { PyFunction pyFunction = (PyFunction) value; // Find the pre-defined output schema TajoDataTypes.Type returnType; - PyObject obj = pyFunction.__findattr__("outputSchema".intern()); + PyObject obj = pyFunction.__findattr__(JythonConstants.SCHEMA_FUNCTION); if (obj != null) { returnType = pyObjectToType(obj); } else { @@ -281,7 +281,6 @@ public static Set registerFunctions(String path, String namespace) } int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; - LOG.info("co_argcount: " + paramNum); TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; for (int i = 0; i < paramNum; i++) { paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); @@ -307,7 +306,6 @@ private static TajoDataTypes.Type pyObjectToType(PyObject obj) { } private static String pyObjectToTypeStringCand(PyObject obj) { - LOG.info("outputSchema: " + obj.toString()); String[] types = obj.toString().split(","); if (types.length > 1) { throw new UnsupportedException("Multiple return type is not supported"); From c2fd9b70ba9f5d57514db5ad4ea8316a94f99b32 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Mar 2015 18:54:14 +0900 Subject: [PATCH 15/55] Add more comments --- .../tajo/function/PythonInvocationDesc.java | 4 +++ .../java/org/apache/tajo/datum/AnyDatum.java | 6 +++- .../plan/expr/GeneralPythonFunctionEval.java | 6 ++++ .../function/python/JythonScriptEngine.java | 3 +- .../plan/function/python/JythonUtils.java | 31 +++++++++++++++++++ 5 files changed, 48 insertions(+), 2 deletions(-) diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java index 8217cd7e15..a2aa0eae83 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java @@ -22,6 +22,10 @@ import org.apache.tajo.catalog.proto.CatalogProtos.PythonInvocationDescProto; import org.apache.tajo.common.ProtoObject; +/** + * PythonInvocationDesc describes a function name + * and a file path to the script where the function is defined. + */ public class PythonInvocationDesc implements ProtoObject { @Expose private String funcName; @Expose private String filePath; diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java index b0deb3a446..0771a6e54f 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/AnyDatum.java @@ -22,6 +22,9 @@ import static org.apache.tajo.common.TajoDataTypes.Type.ANY; +/** + * AnyDatum can contain any types of datum. + */ public class AnyDatum extends Datum { @Expose Datum val; @@ -68,7 +71,8 @@ public int compareTo(Datum datum) { AnyDatum other = (AnyDatum) datum; return val.compareTo(other.val); } - return -1; + // Any datums will be lastly appeared. + return 1; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index b50d37e61e..1d1cec11bd 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -32,6 +32,9 @@ import java.io.IOException; +/** + * Python implementation of a Tajo UDF Performs mappings between Python & Tajo data structures. + */ public class GeneralPythonFunctionEval extends FunctionEval { private Tuple params = null; @@ -53,11 +56,14 @@ public Datum eval(Schema schema, Tuple tuple) { PythonInvocationDesc invokeDesc = funcDesc.getInvocation().getPython(); try { PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); + TajoDataTypes.DataType[] paramTypes = funcDesc.getSignature().getParamTypes(); PyObject result; if (paramTypes.length == 0) { result = function.__call__(); } else { + // Find the actual data types from the given parameters at runtime, + // and convert them into PyObject instances. PyObject[] pyParams = JythonUtils.tupleToPyTuple(params).getArray(); result = function.__call__(pyParams); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 93a9f58cdb..3a00060796 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -272,7 +272,7 @@ public static Set registerFunctions(String path, String namespace) // Find the pre-defined output schema TajoDataTypes.Type returnType; - PyObject obj = pyFunction.__findattr__(JythonConstants.SCHEMA_FUNCTION); + PyObject obj = pyFunction.__findattr__(JythonConstants.OUTPUT_SCHEMA); if (obj != null) { returnType = pyObjectToType(obj); } else { @@ -280,6 +280,7 @@ public static Set registerFunctions(String path, String namespace) returnType = TajoDataTypes.Type.BLOB; } + // Parameters have the ANY type. int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; for (int i = 0; i < paramNum; i++) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java index a68571ad44..130c08c674 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java @@ -30,6 +30,11 @@ public class JythonUtils { + /** + * Convert a datum to a PyObject. + * @param v + * @return + */ public static PyObject datumToPyObject(Datum v) { Preconditions.checkArgument(v.type() == TajoDataTypes.Type.ANY); Datum actual = ((AnyDatum) v).getActual(); @@ -71,6 +76,11 @@ public static PyObject datumToPyObject(Datum v) { } } + /** + * Convert a Tajo tuple to a PyTuple + * @param tuple + * @return + */ public static PyTuple tupleToPyTuple(Tuple tuple) { PyObject[] pyTuple = new PyObject[tuple.size()]; int i = 0; @@ -80,6 +90,11 @@ public static PyTuple tupleToPyTuple(Tuple tuple) { return new PyTuple(pyTuple); } + /** + * Convert a PyObject to a datum. + * @param object + * @return + */ public static Datum pyObjectToDatum(PyObject object) { if (object instanceof PyLong) { return DatumFactory.createInt8((Long) object.__tojava__(Long.class)); @@ -112,10 +127,21 @@ public static Datum pyObjectToDatum(PyObject object) { } } + /** + * Convert a pyObject to a datum of the given type. + * @param object an object will be converted to a datum. + * @param type target datum type. + * @return a datum of the given type. + */ public static Datum pyObjectToDatum(PyObject object, TajoDataTypes.Type type) { return DatumFactory.cast(pyObjectToDatum(object), CatalogUtil.newSimpleDataType(type), null); } + /** + * Convert the primitive type to the Tajo type. + * @param clazz + * @return + */ public static TajoDataTypes.Type primitiveTypeToDataType(Class clazz) { if (clazz.getName().equals(Long.class.getName())) { return TajoDataTypes.Type.INT8; @@ -141,6 +167,11 @@ public static TajoDataTypes.Type primitiveTypeToDataType(Class clazz) { } } + /** + * Convert the Tajo type to the primitive type. + * @param type + * @return + */ public static Object dataTypeToPrimitiveType(TajoDataTypes.Type type) { switch (type) { case BOOLEAN: From b652351fb27001f94faa3e79afc56974f093df9d Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 27 Mar 2015 23:23:09 +0900 Subject: [PATCH 16/55] Add more tests, rename some variables, and add documentations --- .../tajo/engine/function/FunctionLoader.java | 1 - .../org/apache/tajo/TajoTestingCluster.java | 2 +- .../engine/function/TestPythonFunctions.java | 5 +- .../tajo/engine/query/TestSelectQuery.java | 7 ++ .../src/test/resources/python/tajo_util.py | 2 +- .../src/test/resources/python/test_funcs.py | 12 ++-- .../src/test/resources/python/test_funcs2.py | 30 ++++++++ .../testSelectWithPredicateOnPythonFunc.sql | 1 + ...testSelectWithPredicateOnPythonFunc.result | 17 +++++ tajo-docs/src/main/sphinx/functions.rst | 70 ++++++++++++++++++- .../plan/function/python/JythonConstants.java | 2 +- .../function/python/JythonScriptEngine.java | 16 ++--- 12 files changed, 142 insertions(+), 23 deletions(-) create mode 100644 tajo-core/src/test/resources/python/test_funcs2.py create mode 100644 tajo-core/src/test/resources/queries/TestSelectQuery/testSelectWithPredicateOnPythonFunc.sql create mode 100644 tajo-core/src/test/resources/results/TestSelectQuery/testSelectWithPredicateOnPythonFunc.result diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index 0477965427..fee066f5d1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -98,7 +98,6 @@ public static Map loadOptionalFunctions(TajoCon for (FunctionDesc f : JythonScriptEngine.registerFunctions(filePath.toString(), FunctionLoader.PYTHON_FUNCTION_NAMESPACE)) { functionMap.put(f.getSignature(), f); - LOG.info(f); } } } diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 1bf04b250b..1c416d27f9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -165,7 +165,7 @@ void initPropertiesAndConfigs() { // Memory cache termination conf.setIntVar(ConfVars.WORKER_HISTORY_EXPIRE_PERIOD, 1); - conf.setStrings(ConfVars.PYTHON_CODE_DIR.varname, "python/test_funcs.py"); + conf.setStrings(ConfVars.PYTHON_CODE_DIR.varname, "python/test_funcs.py", "python/test_funcs2.py"); this.standbyWorkerMode = conf.getVar(ConfVars.RESOURCE_MANAGER_CLASS) .indexOf(TajoWorkerResourceManager.class.getName()) >= 0; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java index 1f36832b39..555b2cede2 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -28,8 +28,11 @@ public class TestPythonFunctions extends ExprTestBase { @Test public void test() throws IOException { testSimpleEval("select sum_py(1,2)", new String[]{"3"}); - testSimpleEval("select return_one()", new String[]{"100"}); + testSimpleEval("select return_one()", new String[]{"1"}); testSimpleEval("select helloworld()", new String[]{"Hello, World"}); testSimpleEval("select concat_py('1')", new String[]{"11"}); + testSimpleEval("select comma_format(12345)", new String[]{"12,345"}); + testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); + testSimpleEval("select percent(386, 1000)", new String[]{"38.6"}); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 03916939e4..0a3f19926b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -715,4 +715,11 @@ public void testSelectPythonFuncs() throws Exception { assertResultSet(res); cleanupQuery(res); } + + @Test + public void testSelectWithPredicateOnPythonFunc() throws Exception { + ResultSet res = executeQuery(); + assertResultSet(res); + cleanupQuery(res); + } } \ No newline at end of file diff --git a/tajo-core/src/test/resources/python/tajo_util.py b/tajo-core/src/test/resources/python/tajo_util.py index 63ff5b945c..27c17067e6 100644 --- a/tajo-core/src/test/resources/python/tajo_util.py +++ b/tajo-core/src/test/resources/python/tajo_util.py @@ -16,7 +16,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -def outputSchema(schema_str): +def outputType(type_str): def wrap(f): def wrapped_f(*args): return f(*args) diff --git a/tajo-core/src/test/resources/python/test_funcs.py b/tajo-core/src/test/resources/python/test_funcs.py index 8eb16169b9..89b13ec154 100644 --- a/tajo-core/src/test/resources/python/test_funcs.py +++ b/tajo-core/src/test/resources/python/test_funcs.py @@ -1,5 +1,3 @@ -#!/usr/bin/python - ############################################################################ # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -16,13 +14,11 @@ # See the License for the specific language governing permissions and # limitations under the License. -# from tajo_util import outputSchema - -@outputSchema('int4') +@outputType('int4') def return_one(): - return 100 + return 1 -@outputSchema("text") +@outputType("text") def helloworld(): return 'Hello, World' @@ -30,6 +26,6 @@ def helloworld(): def concat_py(str): return str+str -@outputSchema('int4') +@outputType('int4') def sum_py(a,b): return a+b diff --git a/tajo-core/src/test/resources/python/test_funcs2.py b/tajo-core/src/test/resources/python/test_funcs2.py new file mode 100644 index 0000000000..1565d18469 --- /dev/null +++ b/tajo-core/src/test/resources/python/test_funcs2.py @@ -0,0 +1,30 @@ +############################################################################ +# 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. + +#Percent- Percentage +@outputType("float8") +def percent(num, total): + return num * 100 / float(total) + +#commaFormat- format a number with commas, 12345-> 12,345 +@outputType("text") +def comma_format(num): + return '{:,}'.format(num) + +#concatMultiple- concat multiple words +@outputType("text") +def concat4(word1, word2, word3, word4): + return word1 + " " + word2 + " " + word3 + " " + word4 \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testSelectWithPredicateOnPythonFunc.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testSelectWithPredicateOnPythonFunc.sql new file mode 100644 index 0000000000..d2c5082e4a --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testSelectWithPredicateOnPythonFunc.sql @@ -0,0 +1 @@ +select * from nation where sum_py(n_regionkey,1) > 2 \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testSelectWithPredicateOnPythonFunc.result b/tajo-core/src/test/resources/results/TestSelectQuery/testSelectWithPredicateOnPythonFunc.result new file mode 100644 index 0000000000..6d533b4d9a --- /dev/null +++ b/tajo-core/src/test/resources/results/TestSelectQuery/testSelectWithPredicateOnPythonFunc.result @@ -0,0 +1,17 @@ +n_nationkey,n_name,n_regionkey,n_comment +------------------------------- +4,EGYPT,4,y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d +6,FRANCE,3,refully final requests. regular, ironi +7,GERMANY,3,l platelets. regular accounts x-ray: unusual, regular acco +8,INDIA,2,ss excuses cajole slyly across the packages. deposits print aroun +9,INDONESIA,2, slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull +10,IRAN,4,efully alongside of the slyly final dependencies. +11,IRAQ,4,nic deposits boost atop the quickly final requests? quickly regula +12,JAPAN,2,ously. final, express gifts cajole a +13,JORDAN,4,ic deposits are blithely about the carefully regular pa +18,CHINA,2,c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos +19,ROMANIA,3,ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account +20,SAUDI ARABIA,4,ts. silent requests haggle. closely express packages sleep across the blithely +21,VIETNAM,2,hely enticingly express accounts. even, final +22,RUSSIA,3, requests against the platelets use never according to the quickly regular pint +23,UNITED KINGDOM,3,eans boost carefully special requests. accounts are. carefull \ No newline at end of file diff --git a/tajo-docs/src/main/sphinx/functions.rst b/tajo-docs/src/main/sphinx/functions.rst index fb93d1ebd0..935d0a1561 100644 --- a/tajo-docs/src/main/sphinx/functions.rst +++ b/tajo-docs/src/main/sphinx/functions.rst @@ -2,10 +2,78 @@ Functions ****************** +Tajo provides extensive supports for functions. It includes a lot of built-in functions and user-defined functions which is implemented in Python. + +=================== +Built-in Functions +=================== + .. toctree:: :maxdepth: 1 functions/math_func_and_operators functions/string_func_and_operators functions/datetime_func_and_operators - functions/network_func_and_operators \ No newline at end of file + functions/network_func_and_operators + +============================== +Python User-defined Functions +============================== + +----------------------- +Function registration +----------------------- + +To register Python UDFs, you must install script files in all cluster nodes. +After that, you can register your functions by specifying the paths to those script files in ``tajo-site.xml``. Here is an example of the configuration. + +.. code-block:: xml + + + tajo.function.python.code-dir + /path/to/script1.py,/path/to/script2.py + + +Please note that you can specify multiple paths with ``','`` as a delimiter. Each file can contain multiple functions. Here is a typical example of a script file. + +.. code-block:: python + + # /path/to/script1.py + + @outputSchema('int4') + def return_one(): + return 1 + + @outputSchema("text") + def helloworld(): + return 'Hello, World' + + # No decorator - bytearray + def concat_py(str): + return str+str + + @outputSchema('int4') + def sum_py(a,b): + return a+b + +If the configuration is set properly, every function in the script files are registered when the Tajo cluster starts up. + +----------------------- +Decorators and types +----------------------- + +By default, every function has a return type of ``BLOB``. +You can use Python decorators to define output types for the script functions. Tajo can figure out return types from the annotations of the Python script. + +* ``outputType``: Defines the return data type for a script UDF in a format that Tajo can understand. The defined type must be one of the types supported by Tajo. For supported types, please refer to :doc:`/sql_language/data_model`. + +----------------------- +Query example +----------------------- + +Once the Python UDFs are successfully registered, you can use them as other built-in functions. + +.. code-block:: sql + + default> select concat_py(n_name)::text from nation where sum_py(n_regionkey,1) > 2; + diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java index 5f980581c2..0401d98d74 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java @@ -20,7 +20,7 @@ public class JythonConstants { public final static String SKIP_TOKEN = "__"; - public final static String OUTPUT_SCHEMA = "outputSchema".intern(); + public final static String OUTPUT_TYPE = "outputType".intern(); public final static String OUTPUT_SCHEMA_FUNCTION = "outputSchemaFunction".intern(); public final static String SCHEMA_FUNCTION = "schemaFunction".intern(); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 3a00060796..6940b963d3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -114,19 +114,17 @@ private static class Interpreter { */ static synchronized void init(boolean initPhase, String path) throws IOException { // Decorators - - // "schemaFunction" - // "outputSchema" - // "outputSchemaFunction" + // "outputType" if (!filesLoaded.contains(path)) { - // attempt addition of schema decorator handler, fail silently - interpreter.exec("def outputSchema(schema_def):\n" + // attempt addition of type decorator handler, fail silently + interpreter.exec("def outputType(type_def):\n" + " def decorator(func):\n" - + " func.outputSchema = schema_def\n" + + " func.outputType = type_def\n" + " return func\n" + " return decorator\n\n"); - // TODO: Currently, we don't support the customized output schema feature. + // TODO: Currently, we don't support the customized output type feature. // interpreter.exec("def outputSchemaFunction(schema_def):\n" // + " def decorator(func):\n" // + " func.outputSchemaFunction = schema_def\n" @@ -264,7 +262,7 @@ public static Set registerFunctions(String path, String namespace) String key = (String) item.get(0); Object value = item.get(1); if (!key.startsWith(JythonConstants.SKIP_TOKEN) && !key.equals(JythonConstants.SCHEMA_FUNCTION) - && !key.equals(JythonConstants.OUTPUT_SCHEMA) + && !key.equals(JythonConstants.OUTPUT_TYPE) && !key.equals(JythonConstants.OUTPUT_SCHEMA_FUNCTION) && (value instanceof PyFunction) && (((PyFunction)value).__findattr__(JythonConstants.SCHEMA_FUNCTION)== null)) { @@ -272,7 +270,7 @@ public static Set registerFunctions(String path, String namespace) // Find the pre-defined output schema TajoDataTypes.Type returnType; - PyObject obj = pyFunction.__findattr__(JythonConstants.OUTPUT_SCHEMA); + PyObject obj = pyFunction.__findattr__(JythonConstants.OUTPUT_TYPE); if (obj != null) { returnType = pyObjectToType(obj); } else { From 422f280ecd92d88705c159b2822c49025805af99 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 29 Mar 2015 01:20:25 +0900 Subject: [PATCH 17/55] Fix test failure --- tajo-core/src/test/resources/python/test_funcs.py | 2 +- tajo-docs/src/main/sphinx/functions.rst | 8 ++++---- tajo-plan/pom.xml | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tajo-core/src/test/resources/python/test_funcs.py b/tajo-core/src/test/resources/python/test_funcs.py index 89b13ec154..c816232fc6 100644 --- a/tajo-core/src/test/resources/python/test_funcs.py +++ b/tajo-core/src/test/resources/python/test_funcs.py @@ -22,7 +22,7 @@ def return_one(): def helloworld(): return 'Hello, World' -# No decorator - bytearray +# No decorator - blob def concat_py(str): return str+str diff --git a/tajo-docs/src/main/sphinx/functions.rst b/tajo-docs/src/main/sphinx/functions.rst index 935d0a1561..a59bd70c01 100644 --- a/tajo-docs/src/main/sphinx/functions.rst +++ b/tajo-docs/src/main/sphinx/functions.rst @@ -40,19 +40,19 @@ Please note that you can specify multiple paths with ``','`` as a delimiter. Eac # /path/to/script1.py - @outputSchema('int4') + @outputType('int4') def return_one(): return 1 - @outputSchema("text") + @outputType("text") def helloworld(): return 'Hello, World' - # No decorator - bytearray + # No decorator - blob def concat_py(str): return str+str - @outputSchema('int4') + @outputType('int4') def sum_py(a,b): return a+b diff --git a/tajo-plan/pom.xml b/tajo-plan/pom.xml index d0a50e6c83..fabae4b6c6 100644 --- a/tajo-plan/pom.xml +++ b/tajo-plan/pom.xml @@ -188,7 +188,7 @@ org.python jython-standalone - 2.7-b3 + 2.7-b4 From 14fde9a21a16e182908c4c94e0bccfa55f56bf9b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 29 Mar 2015 01:30:27 +0900 Subject: [PATCH 18/55] Fix findbugs warning --- .../org/apache/tajo/catalog/store/AbstractDBStore.java | 7 +++---- .../java/org/apache/tajo/util/datetime/DateTimeFormat.java | 2 -- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java index 518b499e62..22ef164afc 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java @@ -62,8 +62,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo private Connection conn; - protected Map baseTableMaps = new HashMap(); - protected XMLCatalogSchemaManager catalogSchemaManager; protected abstract String getCatalogDriverName(); @@ -1176,7 +1174,7 @@ private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) th } public void addPartition(int tableId, CatalogProtos.PartitionDescProto partition) throws CatalogException { - Connection conn = null; + Connection conn; PreparedStatement pstmt = null; final String ADD_PARTITION_SQL = "INSERT INTO " + TB_PARTTIONS @@ -1194,11 +1192,11 @@ public void addPartition(int tableId, CatalogProtos.PartitionDescProto partition conn = getConnection(); pstmt = conn.prepareStatement(ADD_PARTITION_SQL); - pstmt.setInt(1, tableId); pstmt.setString(2, partition.getPartitionName()); pstmt.setString(3, partition.getPath()); pstmt.executeUpdate(); + pstmt.close(); if (partition.getPartitionKeysCount() > 0) { pstmt = conn.prepareStatement(ADD_PARTITION_KEYS_SQL); @@ -1277,6 +1275,7 @@ private void dropPartition(int tableId, String partitionName) throws CatalogExce pstmt = conn.prepareStatement(sqlDeletePartitionKeys); pstmt.setInt(1, partitionId); pstmt.executeUpdate(); + pstmt.close(); pstmt = conn.prepareStatement(sqlDeletePartition); pstmt.setInt(1, partitionId); diff --git a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeFormat.java b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeFormat.java index c3aa71e90d..798b9c5664 100644 --- a/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeFormat.java +++ b/tajo-common/src/main/java/org/apache/tajo/util/datetime/DateTimeFormat.java @@ -477,7 +477,6 @@ enum FromCharDateMode KeyWord keyword = new KeyWord(); keyword.name = (String)eachKeywordValue[0]; keyword.len = ((Integer)eachKeywordValue[1]).intValue(); - keyword.id = ((DCH_poz)eachKeywordValue[2]).getValue(); keyword.idType = ((DCH_poz)eachKeywordValue[2]); keyword.is_digit = ((Boolean)eachKeywordValue[3]).booleanValue(); keyword.date_mode = (FromCharDateMode)eachKeywordValue[4]; @@ -513,7 +512,6 @@ public KeySuffix(String name, int len, int id, int type) { static class KeyWord { String name; int len; - int id; DCH_poz idType; boolean is_digit; FromCharDateMode date_mode; From 7f162599f530fbfd60ca3562807db8851e25c120 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 29 Mar 2015 20:09:20 +0900 Subject: [PATCH 19/55] TAJO-1344 --- .../java/org/apache/tajo/catalog/store/AbstractDBStore.java | 4 +++- .../org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java index 22ef164afc..d971ff84ae 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java @@ -61,6 +61,8 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo protected final String catalogUri; private Connection conn; + + protected Map baseTableMaps = new HashMap(); protected XMLCatalogSchemaManager catalogSchemaManager; @@ -1174,7 +1176,7 @@ private void addNewColumn(int tableId, CatalogProtos.ColumnProto columnProto) th } public void addPartition(int tableId, CatalogProtos.PartitionDescProto partition) throws CatalogException { - Connection conn; + Connection conn = null; PreparedStatement pstmt = null; final String ADD_PARTITION_SQL = "INSERT INTO " + TB_PARTTIONS diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index 1d1cec11bd..b361de296d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -33,7 +33,7 @@ import java.io.IOException; /** - * Python implementation of a Tajo UDF Performs mappings between Python & Tajo data structures. + * Python implementation of a Tajo UDF performs mappings between Python & Tajo data structures. */ public class GeneralPythonFunctionEval extends FunctionEval { private Tuple params = null; From 34767321066e2117b76809c44871c92823971bb3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 29 Mar 2015 20:49:16 +0900 Subject: [PATCH 20/55] TAJO-1344 --- CHANGES | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGES b/CHANGES index 9895656589..d0a2ad5bf0 100644 --- a/CHANGES +++ b/CHANGES @@ -7,6 +7,8 @@ Release 0.11.0 - unreleased TAJO-1135: Implement queryable virtual table for cluster information. (jihun) + TAJO-1344: Python UDF support. (jihoon) + IMPROVEMENT TAJO-1350: Refactor FilterPushDownRule::visitJoin() into well-defined, From a71142381718f95714d23c39a40bc2897065491c Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 30 Mar 2015 19:20:07 +0900 Subject: [PATCH 21/55] TAJO-1344 --- tajo-plan/pom.xml | 2 +- .../apache/tajo/plan/function/python/JythonScriptEngine.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-plan/pom.xml b/tajo-plan/pom.xml index fabae4b6c6..c8e031cb85 100644 --- a/tajo-plan/pom.xml +++ b/tajo-plan/pom.xml @@ -188,7 +188,7 @@ org.python jython-standalone - 2.7-b4 + 2.5.3 diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 6940b963d3..c307d8260a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -279,7 +279,7 @@ public static Set registerFunctions(String path, String namespace) } // Parameters have the ANY type. - int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; + int paramNum = ((PyBaseCode) pyFunction.func_code).co_argcount; TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; for (int i = 0; i < paramNum; i++) { paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); From c7e496039183289c43c4f55d1be1afffccde9341 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 3 Apr 2015 17:25:45 +0900 Subject: [PATCH 22/55] TAJO-1344 --- .../tajo/function/PythonInvocationDesc.java | 32 +- .../java/org/apache/tajo/storage/Tuple.java | 54 +-- .../resources/python/__init__.py} | 9 +- .../src/main/resources/python/controller.py | 367 ++++++++++++++++++ .../src/main/resources/python/tajo_util.py | 101 +++++ tajo-plan/pom.xml | 2 +- .../org/apache/tajo/plan/ExprAnnotator.java | 6 +- .../tajo/plan/expr/BasicEvalNodeVisitor.java | 8 - .../tajo/plan/expr/EvalNodeVisitor2.java | 1 - .../org/apache/tajo/plan/expr/EvalType.java | 2 - .../apache/tajo/plan/expr/FunctionInvoke.java | 75 ++++ .../tajo/plan/expr/GeneralFunctionEval.java | 24 +- .../plan/expr/GeneralPythonFunctionEval.java | 154 ++++---- .../plan/expr/LegacyScalarFunctionInvoke.java | 69 ++++ .../tajo/plan/expr/PythonFunctionInvoke.java | 97 +++++ .../tajo/plan/expr/PythonFunctionInvoke2.java | 201 ++++++++++ .../tajo/plan/expr/SimpleEvalNodeVisitor.java | 3 - .../function/python/JythonScriptEngine.java | 2 +- .../python/ScriptingOutputCapturer.java | 151 +++++++ .../plan/function/stream/InputHandler.java | 86 ++++ .../plan/function/stream/RowStoreUtil.java | 272 +++++++++++++ .../stream/StreamingUDFException.java | 57 +++ .../plan/serder/EvalNodeDeserializer.java | 6 +- 23 files changed, 1632 insertions(+), 147 deletions(-) rename tajo-core/src/{test/resources/python/tajo_util.py => main/resources/python/__init__.py} (82%) create mode 100644 tajo-core/src/main/resources/python/controller.py create mode 100644 tajo-core/src/main/resources/python/tajo_util.py create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java index a2aa0eae83..f38735ac20 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java @@ -18,15 +18,17 @@ package org.apache.tajo.function; +import com.google.common.base.Objects; import com.google.gson.annotations.Expose; import org.apache.tajo.catalog.proto.CatalogProtos.PythonInvocationDescProto; import org.apache.tajo.common.ProtoObject; +import org.apache.tajo.util.TUtil; /** * PythonInvocationDesc describes a function name * and a file path to the script where the function is defined. */ -public class PythonInvocationDesc implements ProtoObject { +public class PythonInvocationDesc implements ProtoObject, Cloneable { @Expose private String funcName; @Expose private String filePath; @@ -65,4 +67,32 @@ public PythonInvocationDescProto getProto() { builder.setFuncName(funcName).setFilePath(filePath); return builder.build(); } + + @Override + public boolean equals(Object o) { + if (o instanceof PythonInvocationDesc) { + PythonInvocationDesc other = (PythonInvocationDesc) o; + return TUtil.checkEquals(funcName, other.funcName) && + TUtil.checkEquals(filePath, other.filePath); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(funcName, filePath); + } + + @Override + public String toString() { + return funcName + " at " + filePath; + } + + @Override + public Object clone() throws CloneNotSupportedException { + PythonInvocationDesc clone = (PythonInvocationDesc) super.clone(); + clone.funcName = funcName == null ? null : new String(funcName); + clone.filePath = filePath == null ? null : new String(filePath); + return clone; + } } diff --git a/tajo-common/src/main/java/org/apache/tajo/storage/Tuple.java b/tajo-common/src/main/java/org/apache/tajo/storage/Tuple.java index 1ba19266e0..aec784fc52 100644 --- a/tajo-common/src/main/java/org/apache/tajo/storage/Tuple.java +++ b/tajo-common/src/main/java/org/apache/tajo/storage/Tuple.java @@ -22,58 +22,58 @@ public interface Tuple extends Cloneable { - public int size(); + int size(); - public boolean contains(int fieldid); + boolean contains(int fieldid); - public boolean isNull(int fieldid); + boolean isNull(int fieldid); @SuppressWarnings("unused") - public boolean isNotNull(int fieldid); + boolean isNotNull(int fieldid); - public void clear(); + void clear(); - public void put(int fieldId, Datum value); + void put(int fieldId, Datum value); - public void put(int fieldId, Datum[] values); + void put(int fieldId, Datum[] values); - public void put(int fieldId, Tuple tuple); + void put(int fieldId, Tuple tuple); - public void put(Datum[] values); + void put(Datum[] values); - public Datum get(int fieldId); + Datum get(int fieldId); - public void setOffset(long offset); + void setOffset(long offset); - public long getOffset(); + long getOffset(); - public boolean getBool(int fieldId); + boolean getBool(int fieldId); - public byte getByte(int fieldId); + byte getByte(int fieldId); - public char getChar(int fieldId); + char getChar(int fieldId); - public byte [] getBytes(int fieldId); + byte [] getBytes(int fieldId); - public short getInt2(int fieldId); + short getInt2(int fieldId); - public int getInt4(int fieldId); + int getInt4(int fieldId); - public long getInt8(int fieldId); + long getInt8(int fieldId); - public float getFloat4(int fieldId); + float getFloat4(int fieldId); - public double getFloat8(int fieldId); + double getFloat8(int fieldId); - public String getText(int fieldId); + String getText(int fieldId); - public Datum getProtobufDatum(int fieldId); + Datum getProtobufDatum(int fieldId); - public Datum getInterval(int fieldId); + Datum getInterval(int fieldId); - public char [] getUnicodeChars(int fieldId); + char [] getUnicodeChars(int fieldId); - public Tuple clone() throws CloneNotSupportedException; + Tuple clone() throws CloneNotSupportedException; - public Datum[] getValues(); + Datum[] getValues(); } diff --git a/tajo-core/src/test/resources/python/tajo_util.py b/tajo-core/src/main/resources/python/__init__.py similarity index 82% rename from tajo-core/src/test/resources/python/tajo_util.py rename to tajo-core/src/main/resources/python/__init__.py index 27c17067e6..8093a2f8fa 100644 --- a/tajo-core/src/test/resources/python/tajo_util.py +++ b/tajo-core/src/main/resources/python/__init__.py @@ -14,11 +14,4 @@ # 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. - -def outputType(type_str): - def wrap(f): - def wrapped_f(*args): - return f(*args) - return wrapped_f - return wrap +# limitations under the License. \ No newline at end of file diff --git a/tajo-core/src/main/resources/python/controller.py b/tajo-core/src/main/resources/python/controller.py new file mode 100644 index 0000000000..a37e868f4a --- /dev/null +++ b/tajo-core/src/main/resources/python/controller.py @@ -0,0 +1,367 @@ +#! /usr/bin/env python +############################################################################ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import sys +import os +import logging + +from datetime import datetime +try: + from dateutil import parser + USE_DATEUTIL = True +except ImportError: + USE_DATEUTIL = False + +from tajo_util import write_user_exception, udf_logging + +FIELD_DELIMITER = ',' +TUPLE_START = '(' +TUPLE_END = ')' +BAG_START = '{' +BAG_END = '}' +MAP_START = '[' +MAP_END = ']' +MAP_KEY = '#' +PARAMETER_DELIMITER = '\t' +PRE_WRAP_DELIM = '|' +POST_WRAP_DELIM = '_' +NULL_BYTE = "-" +END_RECORD_DELIM = '|_\n' +END_RECORD_DELIM_LENGTH = len(END_RECORD_DELIM) + +WRAPPED_FIELD_DELIMITER = PRE_WRAP_DELIM + FIELD_DELIMITER + POST_WRAP_DELIM +WRAPPED_TUPLE_START = PRE_WRAP_DELIM + TUPLE_START + POST_WRAP_DELIM +WRAPPED_TUPLE_END = PRE_WRAP_DELIM + TUPLE_END + POST_WRAP_DELIM +WRAPPED_BAG_START = PRE_WRAP_DELIM + BAG_START + POST_WRAP_DELIM +WRAPPED_BAG_END = PRE_WRAP_DELIM + BAG_END + POST_WRAP_DELIM +WRAPPED_MAP_START = PRE_WRAP_DELIM + MAP_START + POST_WRAP_DELIM +WRAPPED_MAP_END = PRE_WRAP_DELIM + MAP_END + POST_WRAP_DELIM +WRAPPED_PARAMETER_DELIMITER = PRE_WRAP_DELIM + PARAMETER_DELIMITER + POST_WRAP_DELIM +WRAPPED_NULL_BYTE = PRE_WRAP_DELIM + NULL_BYTE + POST_WRAP_DELIM + +TYPE_TUPLE = TUPLE_START +TYPE_BAG = BAG_START +TYPE_MAP = MAP_START + +TYPE_BOOLEAN = "B" +TYPE_INTEGER = "I" +TYPE_LONG = "L" +TYPE_FLOAT = "F" +TYPE_DOUBLE = "D" +TYPE_BYTEARRAY = "A" +TYPE_CHARARRAY = "C" +TYPE_DATETIME = "T" +TYPE_BIGINTEGER = "N" +TYPE_BIGDECIMAL = "E" + +END_OF_STREAM = TYPE_CHARARRAY + "\x04" + END_RECORD_DELIM +TURN_ON_OUTPUT_CAPTURING = TYPE_CHARARRAY + "TURN_ON_OUTPUT_CAPTURING" + END_RECORD_DELIM +NUM_LINES_OFFSET_TRACE = int(os.environ.get('PYTHON_TRACE_OFFSET', 0)) + +class PythonStreamingController: + def __init__(self, profiling_mode=False): + self.profiling_mode = profiling_mode + + self.input_count = 0 + self.next_input_count_to_log = 1 + + def main(self, + module_name, file_path, func_name, cache_path, + output_stream_path, error_stream_path, log_file_name, is_illustrate_str): + sys.stdin = os.fdopen(sys.stdin.fileno(), 'rb', 0) + + #Need to ensure that user functions can't write to the streams we use to + #communicate with pig. + self.stream_output = os.fdopen(sys.stdout.fileno(), 'wb', 0) + self.stream_error = os.fdopen(sys.stderr.fileno(), 'wb', 0) + + self.input_stream = sys.stdin + self.output_stream = open(output_stream_path, 'a') + sys.stderr = open(error_stream_path, 'w') + is_illustrate = is_illustrate_str == "true" + + sys.path.append(file_path) + sys.path.append(cache_path) + sys.path.append('.') + + logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) + logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") + + input_str = self.get_next_input() + + try: + func = __import__(module_name, globals(), locals(), [func_name], -1).__dict__[func_name] + except: + #These errors should always be caused by user code. + write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) + self.close_controller(-1) + + if is_illustrate or udf_logging.udf_log_level != logging.DEBUG: + #Only log output for illustrate after we get the flag to capture output. + sys.stdout = open(os.devnull, 'w') + else: + sys.stdout = self.output_stream + + while input_str != END_OF_STREAM: + should_log = False + if self.input_count == self.next_input_count_to_log: + should_log = True + log_message = logging.info + self.update_next_input_count_to_log() + elif udf_logging.udf_log_level == logging.DEBUG: + should_log = True + log_message = logging.debug + + try: + try: + if should_log: + log_message("Row %s: Serialized Input: %s" % (self.input_count, input_str)) + inputs = deserialize_input(input_str) + if should_log: + log_message("Row %s: Deserialized Input: %s" % (self.input_count, unicode(inputs))) + except: + #Capture errors where the user passes in bad data. + write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) + self.close_controller(-3) + + try: + func_output = func(*inputs) + if should_log: + log_message("Row %s: UDF Output: %s" % (self.input_count, unicode(func_output))) + except: + #These errors should always be caused by user code. + write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) + self.close_controller(-2) + + output = serialize_output(func_output) + if should_log: + log_message("Row %s: Serialized Output: %s" % (self.input_count, output)) + + self.stream_output.write( "%s%s" % (output, END_RECORD_DELIM) ) + except Exception as e: + #This should only catch internal exceptions with the controller + #and pig- not with user code. + import traceback + traceback.print_exc(file=self.stream_error) + sys.exit(-3) + + sys.stdout.flush() + sys.stderr.flush() + self.stream_output.flush() + self.stream_error.flush() + + input_str = self.get_next_input() + + def get_next_input(self): + input_stream = self.input_stream + output_stream = self.output_stream + + input_str = input_stream.readline() + + while input_str.endswith(END_RECORD_DELIM) == False: + line = input_stream.readline() + if line == '': + input_str = '' + break + input_str += line + + if input_str == '': + return END_OF_STREAM + + if input_str == TURN_ON_OUTPUT_CAPTURING: + logging.debug("Turned on Output Capturing") + sys.stdout = output_stream + return self.get_next_input() + + if input_str == END_OF_STREAM: + return input_str + + self.input_count += 1 + + return input_str[:-END_RECORD_DELIM_LENGTH] + + def update_next_input_count_to_log(self): + """ + Want to log enough rows that you can see progress being made and see timings without wasting time logging thousands of rows. + Show first 10 rows, and then the first 5 rows of every order of magnitude (10-15, 100-105, 1000-1005, ...) + """ + if self.next_input_count_to_log < 10: + self.next_input_count_to_log = self.next_input_count_to_log + 1 + elif self.next_input_count_to_log % 10 == 5: + self.next_input_count_to_log = (self.next_input_count_to_log - 5) * 10 + else: + self.next_input_count_to_log = self.next_input_count_to_log + 1 + + def close_controller(self, exit_code): + sys.stderr.close() + self.stream_error.write("\n") + self.stream_error.close() + sys.stdout.close() + self.stream_output.write("\n") + self.stream_output.close() + sys.exit(exit_code) + +def deserialize_input(input_str): + if len(input_str) == 0: + return [] + + return [_deserialize_input(param, 0, len(param)-1) for param in input_str.split(WRAPPED_PARAMETER_DELIMITER)] + +def _deserialize_input(input_str, si, ei): + if ei - si < 1: + #Handle all of the cases where you can have valid empty input. + if ei == si: + if input_str[si] == TYPE_CHARARRAY: + return u"" + elif input_str[si] == TYPE_BYTEARRAY: + return bytearray("") + else: + raise Exception("Got input type flag %s, but no data to go with it.\nInput string: %s\nSlice: %s" % (input_str[si], input_str, input_str[si:ei+1])) + else: + raise Exception("Start index %d greater than end index %d.\nInput string: %s\n, Slice: %s" % (si, ei, input_str[si:ei+1])) + + first = input_str[si] + schema = input_str[si+1] if first == PRE_WRAP_DELIM else first + + if schema == NULL_BYTE: + return None + elif schema == TYPE_TUPLE or schema == TYPE_MAP or schema == TYPE_BAG: + return _deserialize_collection(input_str, schema, si+3, ei-3) + elif schema == TYPE_CHARARRAY: + return unicode(input_str[si+1:ei+1], 'utf-8') + elif schema == TYPE_BYTEARRAY: + return bytearray(input_str[si+1:ei+1]) + elif schema == TYPE_INTEGER: + return int(input_str[si+1:ei+1]) + elif schema == TYPE_LONG or schema == TYPE_BIGINTEGER: + return long(input_str[si+1:ei+1]) + elif schema == TYPE_FLOAT or schema == TYPE_DOUBLE or schema == TYPE_BIGDECIMAL: + return float(input_str[si+1:ei+1]) + elif schema == TYPE_BOOLEAN: + return input_str[si+1:ei+1] == "true" + elif schema == TYPE_DATETIME: + #Format is "yyyy-MM-ddTHH:mm:ss.SSS+00:00" or "2013-08-23T18:14:03.123+ZZ" + if USE_DATEUTIL: + return parser.parse(input_str[si+1:ei+1]) + else: + #Try to use datetime even though it doesn't handle time zones properly, + #We only use the first 3 microsecond digits and drop time zone (first 23 characters) + return datetime.strptime(input_str[si+1:si+24], "%Y-%m-%dT%H:%M:%S.%f") + else: + raise Exception("Can't determine type of input: %s" % input_str[si:ei+1]) + +def _deserialize_collection(input_str, return_type, si, ei): + list_result = [] + append_to_list_result = list_result.append + dict_result = {} + + index = si + field_start = si + depth = 0 + + key = None + + # recurse to deserialize elements if the collection is not empty + if ei-si+1 > 0: + while True: + if index >= ei - 2: + if return_type == TYPE_MAP: + dict_result[key] = _deserialize_input(input_str, value_start, ei) + else: + append_to_list_result(_deserialize_input(input_str, field_start, ei)) + break + + if return_type == TYPE_MAP and not key: + key_index = input_str.find(MAP_KEY, index) + key = unicode(input_str[index+1:key_index], 'utf-8') + index = key_index + 1 + value_start = key_index + 1 + continue + + if not (input_str[index] == PRE_WRAP_DELIM and input_str[index+2] == POST_WRAP_DELIM): + prewrap_index = input_str.find(PRE_WRAP_DELIM, index+1) + index = (prewrap_index if prewrap_index != -1 else end_index) + continue + + mid = input_str[index+1] + + if mid == BAG_START or mid == TUPLE_START or mid == MAP_START: + depth += 1 + elif mid == BAG_END or mid == TUPLE_END or mid == MAP_END: + depth -= 1 + elif depth == 0 and mid == FIELD_DELIMITER: + if return_type == TYPE_MAP: + dict_result[key] = _deserialize_input(input_str, value_start, index - 1) + key = None + else: + append_to_list_result(_deserialize_input(input_str, field_start, index - 1)) + field_start = index + 3 + + index += 3 + + if return_type == TYPE_MAP: + return dict_result + elif return_type == TYPE_TUPLE: + return tuple(list_result) + else: + return list_result + +def wrap_tuple(o, serialized_item): + if type(o) != tuple: + return WRAPPED_TUPLE_START + serialized_item + WRAPPED_TUPLE_END + else: + return serialized_item + +def serialize_output(output, utfEncodeAllFields=False): + """ + @param utfEncodeStrings - Generally we want to utf encode only strings. But for + Maps we utf encode everything because on the Java side we don't know the schema + for maps so we wouldn't be able to tell which fields were encoded or not. + """ + + output_type = type(output) + + if output is None: + return WRAPPED_NULL_BYTE + elif output_type == tuple: + return (WRAPPED_TUPLE_START + + WRAPPED_FIELD_DELIMITER.join([serialize_output(o, utfEncodeAllFields) for o in output]) + + WRAPPED_TUPLE_END) + elif output_type == list: + return (WRAPPED_BAG_START + + WRAPPED_FIELD_DELIMITER.join([wrap_tuple(o, serialize_output(o, utfEncodeAllFields)) for o in output]) + + WRAPPED_BAG_END) + elif output_type == dict: + return (WRAPPED_MAP_START + + WRAPPED_FIELD_DELIMITER.join(['%s%s%s' % (k.encode('utf-8'), MAP_KEY, serialize_output(v, True)) for k, v in output.iteritems()]) + + WRAPPED_MAP_END) + elif output_type == bool: + return ("true" if output else "false") + elif output_type == bytearray: + return str(output) + elif output_type == datetime: + return output.isoformat() + elif utfEncodeAllFields or output_type == str or output_type == unicode: + #unicode is necessary in cases where we're encoding non-strings. + return unicode(output).encode('utf-8') + else: + return str(output) + +if __name__ == '__main__': + controller = PythonStreamingController() + controller.main(sys.argv[1], sys.argv[2], sys.argv[3], sys.argv[4], + sys.argv[5], sys.argv[6], sys.argv[7], sys.argv[8]) diff --git a/tajo-core/src/main/resources/python/tajo_util.py b/tajo-core/src/main/resources/python/tajo_util.py new file mode 100644 index 0000000000..f97e0c5bea --- /dev/null +++ b/tajo-core/src/main/resources/python/tajo_util.py @@ -0,0 +1,101 @@ +#!/usr/bin/python + +############################################################################ +# 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. + +class udf_logging(object): + udf_log_level = logging.INFO + + @classmethod + def set_log_level_error(cls): + cls.udf_log_level = logging.ERROR + + @classmethod + def set_log_level_warn(cls): + cls.udf_log_level = logging.WARN + + @classmethod + def set_log_level_info(cls): + cls.udf_log_level = logging.INFO + + @classmethod + def set_log_level_debug(cls): + cls.udf_log_level = logging.DEBUG + +def outputType(type_str): + def wrap(f): + def wrapped_f(*args): + return f(*args) + return wrapped_f + return wrap + +def write_user_exception(filename, stream_err_output, num_lines_offset_trace=0): + import sys + import traceback + import inspect + (t, v, tb) = sys.exc_info() + name = t.__name__ + record_error = False + + if name in ['SyntaxError', 'IndentationError']: + syntax_error_values = v.args + user_line_number = syntax_error_values[1][1] - num_lines_offset_trace + error_message = "%s: %s\n\tFile: %s, line %s column %s\n\t%s" % \ + (name, + syntax_error_values[0], + syntax_error_values[1][0], + user_line_number, + syntax_error_values[1][2], + syntax_error_values[1][3]) + else: + error_message = "%s: %s\n" % (name, v) + user_line_number = None + while 1: + e_file_name = tb.tb_frame.f_code.co_filename + if e_file_name.find(filename) > 0: + record_error = True + if not record_error: + if not tb.tb_next: + break + tb = tb.tb_next + continue + + line_number = tb.tb_lineno + mod = inspect.getmodule(tb) + if mod: + lines, offset = inspect.getsourcelines(mod) + line = lines[line_number - offset - 1] + else: + #Useful to catch exceptions with an invalid module (like syntax + #errors) + lines, offset = inspect.getsourcelines(tb.tb_frame) + if (line_number - 1) >= len(lines): + line = "Unknown Line" + else: + line = lines[line_number - 1] + + user_line_number = line_number - num_lines_offset_trace + func_name = tb.tb_frame.f_code.co_name + error_message += 'File %s, line %s, in %s\n\t%s\n' % \ + (e_file_name, user_line_number, func_name, line) + if not tb.tb_next: + break + tb = tb.tb_next + if name in ['UnicodeEncodeError']: + error_message += "\nTo print a unicode string in your udf use encode('utf-8'). Example: \n\tprint 'Example'.encode('utf-8')" + if user_line_number: + stream_err_output.write("%s\n" % user_line_number) + stream_err_output.write("%s\n" % error_message) diff --git a/tajo-plan/pom.xml b/tajo-plan/pom.xml index c8e031cb85..a7789c0113 100644 --- a/tajo-plan/pom.xml +++ b/tajo-plan/pom.xml @@ -188,7 +188,7 @@ org.python jython-standalone - 2.5.3 + 2.7-b1 diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index 1c88ea77dc..b5d883136b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -608,11 +608,7 @@ public EvalNode visitFunction(Context ctx, Stack stack, FunctionExpr expr) FunctionType functionType = funcDesc.getFuncType(); if (functionType == FunctionType.GENERAL || functionType == FunctionType.UDF) { - if (funcDesc.getInvocation().hasPython()) { - return new GeneralPythonFunctionEval(funcDesc, givenArgs); - } else { - return new GeneralFunctionEval(ctx.queryContext, funcDesc, (GeneralFunction) funcDesc.newInstance(), givenArgs); - } + return new GeneralFunctionEval(ctx.queryContext, funcDesc, givenArgs); } else if (functionType == FunctionType.AGGREGATION || functionType == FunctionType.UDA) { if (!ctx.currentBlock.hasNode(NodeType.GROUP_BY)) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java index 4620698429..81b0f8e9dc 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BasicEvalNodeVisitor.java @@ -128,9 +128,6 @@ public RESULT visitChild(CONTEXT context, EvalNode evalNode, Stack sta case WINDOW_FUNCTION: result = visitWindowFunc(context, (WindowFunctionEval) evalNode, stack); break; - case PYTHON_FUNCTION: - result = visitPythonFunc(context, (GeneralPythonFunctionEval) evalNode, stack); - break; case SIGNED: result = visitSigned(context, (SignedEval) evalNode, stack); @@ -336,11 +333,6 @@ public RESULT visitWindowFunc(CONTEXT context, WindowFunctionEval evalNode, Stac return visitDefaultFunctionEval(context, evalNode, stack); } - @Override - public RESULT visitPythonFunc(CONTEXT context, GeneralPythonFunctionEval evalNode, Stack stack) { - return visitDefaultFunctionEval(context, evalNode, stack); - } - @Override public RESULT visitSigned(CONTEXT context, SignedEval signedEval, Stack stack) { return visitDefaultUnaryEval(context, signedEval, stack); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java index 99a66d7f77..bae193abde 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNodeVisitor2.java @@ -65,7 +65,6 @@ public interface EvalNodeVisitor2 { RESULT visitFuncCall(CONTEXT context, GeneralFunctionEval evalNode, Stack stack); RESULT visitAggrFuncCall(CONTEXT context, AggregationFunctionCallEval evalNode, Stack stack); RESULT visitWindowFunc(CONTEXT context, WindowFunctionEval evalNode, Stack stack); - RESULT visitPythonFunc(CONTEXT context, GeneralPythonFunctionEval evalNode, Stack stack); RESULT visitSigned(CONTEXT context, SignedEval signedEval, Stack stack); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java index 5d51d47d5f..c1df658a1b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalType.java @@ -47,7 +47,6 @@ public enum EvalType { WINDOW_FUNCTION(WindowFunctionEval.class), AGG_FUNCTION(AggregationFunctionCallEval.class), FUNCTION(GeneralFunctionEval.class), - PYTHON_FUNCTION(GeneralPythonFunctionEval.class), // String operator or pattern matching predicates LIKE(LikePredicateEval.class), @@ -149,7 +148,6 @@ public static boolean isFunction(EvalType type) { match |= type == FUNCTION; match |= type == AGG_FUNCTION; match |= type == WINDOW_FUNCTION; - match |= type == PYTHON_FUNCTION; return match; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java new file mode 100644 index 0000000000..66ce6293b2 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.expr; + +import com.google.gson.annotations.Expose; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.exception.InternalException; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.storage.Tuple; + +public abstract class FunctionInvoke { + @Expose protected FunctionDesc functionDesc; + + public FunctionInvoke(FunctionDesc functionDesc) { + this.functionDesc = functionDesc; + } + + public static FunctionInvoke newInstance(FunctionDesc desc) throws InternalException { + if (desc.getInvocation().hasLegacy()) { + return new LegacyScalarFunctionInvoke(desc); + } else if (desc.getInvocation().hasPython()) { + return new PythonFunctionInvoke(desc); + } else { + throw new UnsupportedException(desc.getInvocation() + " is not supported"); + } + } + + public abstract void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes); + + /** + * Evaluate the given tuple with a function + * @param tuple a tuple evaluated with parameters + * @return a result of a fuction execution + */ + public abstract Datum eval(Tuple tuple); + + @Override + public boolean equals(Object o) { + if (o instanceof FunctionInvoke) { + FunctionInvoke other = (FunctionInvoke) o; + return this.functionDesc.equals(other.functionDesc); + } + return false; + } + + @Override + public int hashCode() { + return functionDesc.hashCode(); + } + + @Override + public Object clone() throws CloneNotSupportedException { + FunctionInvoke clone = (FunctionInvoke) super.clone(); + clone.functionDesc = (FunctionDesc) this.functionDesc.clone(); + return clone; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 747e8ba19d..462ddc81fd 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -22,7 +22,9 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.OverridableConf; import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; +import org.apache.tajo.exception.InternalException; import org.apache.tajo.plan.function.GeneralFunction; import org.apache.tajo.storage.Tuple; import org.apache.tajo.util.TUtil; @@ -30,19 +32,23 @@ import javax.annotation.Nullable; public class GeneralFunctionEval extends FunctionEval { - @Expose protected GeneralFunction instance; + @Expose protected FunctionInvoke funcInvoke; - public GeneralFunctionEval(@Nullable OverridableConf queryContext, FunctionDesc desc, GeneralFunction instance, - EvalNode[] givenArgs) { + public GeneralFunctionEval(@Nullable OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) throws InternalException { super(EvalType.FUNCTION, desc, givenArgs); - this.instance = instance; - this.instance.init(queryContext, getParamType()); + this.funcInvoke = FunctionInvoke.newInstance(desc); + this.funcInvoke.init(queryContext, getParamType()); + } + + @Override + public void bind(Schema schema) { + super.bind(schema); } @Override @SuppressWarnings("unchecked") public Datum eval(Tuple tuple) { - return instance.eval(evalParams(tuple)); + return funcInvoke.eval(evalParams(tuple)); } @Override @@ -50,7 +56,7 @@ public boolean equals(Object obj) { if (obj instanceof GeneralFunctionEval) { GeneralFunctionEval other = (GeneralFunctionEval) obj; return super.equals(other) && - TUtil.checkEquals(instance, other.instance); + TUtil.checkEquals(funcInvoke, other.funcInvoke); } return false; @@ -58,13 +64,13 @@ public boolean equals(Object obj) { @Override public int hashCode() { - return Objects.hashCode(funcDesc, instance); + return Objects.hashCode(funcDesc, funcInvoke); } @Override public Object clone() throws CloneNotSupportedException { GeneralFunctionEval eval = (GeneralFunctionEval) super.clone(); - eval.instance = (GeneralFunction) instance.clone(); + eval.funcInvoke = (FunctionInvoke) funcInvoke.clone(); return eval; } } \ No newline at end of file diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java index b361de296d..05453dc9e5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java @@ -1,76 +1,78 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.expr; - -import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.function.PythonInvocationDesc; -import org.apache.tajo.plan.function.python.JythonScriptEngine; -import org.apache.tajo.plan.function.python.JythonUtils; -import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; -import org.python.core.PyFunction; -import org.python.core.PyObject; - -import java.io.IOException; - -/** - * Python implementation of a Tajo UDF performs mappings between Python & Tajo data structures. - */ -public class GeneralPythonFunctionEval extends FunctionEval { - private Tuple params = null; - - public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { - super(EvalType.PYTHON_FUNCTION, funcDesc, argEvals); - } - - @Override - public Datum eval(Schema schema, Tuple tuple) { - if (this.params == null) { - params = new VTuple(argEvals.length); - } - if(argEvals != null) { - params.clear(); - for(int i=0;i < argEvals.length; i++) { - params.put(i, argEvals[i].eval(schema, tuple)); - } - } - PythonInvocationDesc invokeDesc = funcDesc.getInvocation().getPython(); - try { - PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); - - TajoDataTypes.DataType[] paramTypes = funcDesc.getSignature().getParamTypes(); - PyObject result; - if (paramTypes.length == 0) { - result = function.__call__(); - } else { - // Find the actual data types from the given parameters at runtime, - // and convert them into PyObject instances. - PyObject[] pyParams = JythonUtils.tupleToPyTuple(params).getArray(); - result = function.__call__(pyParams); - } - - return JythonUtils.pyObjectToDatum(result); - } catch (IOException e) { - throw new RuntimeException(e); - } - } -} +///** +// * Licensed to the Apache Software Foundation (ASF) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. The ASF licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// +//package org.apache.tajo.plan.expr; +// +//import org.apache.tajo.catalog.FunctionDesc; +//import org.apache.tajo.catalog.Schema; +//import org.apache.tajo.common.TajoDataTypes; +//import org.apache.tajo.datum.Datum; +//import org.apache.tajo.function.PythonInvocationDesc; +//import org.apache.tajo.plan.function.python.JythonScriptEngine; +//import org.apache.tajo.plan.function.python.JythonUtils; +//import org.apache.tajo.storage.Tuple; +//import org.apache.tajo.storage.VTuple; +//import org.python.core.PyFunction; +//import org.python.core.PyObject; +// +//import java.io.IOException; +// +///** +// * Python implementation of a Tajo UDF performs mappings between Python & Tajo data structures. +// */ +//public class GeneralPythonFunctionEval extends FunctionEval { +// private Tuple params = null; +// +// public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { +// super(EvalType.PYTHON_FUNCTION, funcDesc, argEvals); +// } +// +// @Override +// public Datum eval(Schema schema, Tuple tuple) { +// if (this.params == null) { +// params = new VTuple(argEvals.length); +// } +// if(argEvals != null) { +// params.clear(); +// for(int i=0;i < argEvals.length; i++) { +// params.put(i, argEvals[i].eval(schema, tuple)); +// } +// } +// +// +// PythonInvocationDesc invokeDesc = funcDesc.getInvocation().getPython(); +// try { +// PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); +// +// TajoDataTypes.DataType[] paramTypes = funcDesc.getSignature().getParamTypes(); +// PyObject result; +// if (paramTypes.length == 0) { +// result = function.__call__(); +// } else { +// // Find the actual data types from the given parameters at runtime, +// // and convert them into PyObject instances. +// PyObject[] pyParams = JythonUtils.tupleToPyTuple(params).getArray(); +// result = function.__call__(pyParams); +// } +// +// return JythonUtils.pyObjectToDatum(result); +// } catch (IOException e) { +// throw new RuntimeException(e); +// } +// } +//} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java new file mode 100644 index 0000000000..ca1244d0bc --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.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.tajo.plan.expr; + +import com.google.gson.annotations.Expose; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.exception.InternalException; +import org.apache.tajo.plan.function.GeneralFunction; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.util.TUtil; + +public class LegacyScalarFunctionInvoke extends FunctionInvoke { + @Expose private GeneralFunction function; + + public LegacyScalarFunctionInvoke(FunctionDesc funcDesc) throws InternalException { + super(funcDesc); + function = (GeneralFunction) funcDesc.newInstance(); + } + + @Override + public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { + function.init(queryContext, paramTypes); + } + + @Override + public Datum eval(Tuple tuple) { + return function.eval(tuple); + } + + @Override + public boolean equals(Object o) { + if (o instanceof LegacyScalarFunctionInvoke) { + LegacyScalarFunctionInvoke other = (LegacyScalarFunctionInvoke) o; + return super.equals(other) && + TUtil.checkEquals(function, other.function); + } + return false; + } + + @Override + public int hashCode() { + return function.hashCode(); + } + + @Override + public Object clone() throws CloneNotSupportedException { + LegacyScalarFunctionInvoke clone = (LegacyScalarFunctionInvoke) super.clone(); + clone.function = (GeneralFunction) function.clone(); + return clone; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java new file mode 100644 index 0000000000..e083824023 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java @@ -0,0 +1,97 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.expr; + +import com.google.common.base.Objects; +import com.google.gson.annotations.Expose; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.function.PythonInvocationDesc; +import org.apache.tajo.plan.function.python.JythonScriptEngine; +import org.apache.tajo.plan.function.python.JythonUtils; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.util.TUtil; +import org.python.core.PyFunction; +import org.python.core.PyObject; + +import java.io.IOException; +import java.util.Arrays; + +public class PythonFunctionInvoke extends FunctionInvoke { + @Expose private PythonInvocationDesc invokeDesc; + @Expose private TajoDataTypes.DataType[] paramTypes; + + public PythonFunctionInvoke(FunctionDesc funcDesc) { + super(funcDesc); + this.invokeDesc = funcDesc.getInvocation().getPython(); + this.paramTypes = funcDesc.getSignature().getParamTypes(); + } + + @Override + public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { + // nothing to do + } + + @Override + public Datum eval(Tuple tuple) { + try { + PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); + + PyObject result; + if (paramTypes.length == 0) { + result = function.__call__(); + } else { + // Find the actual data types from the given parameters at runtime, + // and convert them into PyObject instances. + PyObject[] pyParams = JythonUtils.tupleToPyTuple(tuple).getArray(); + result = function.__call__(pyParams); + } + + return JythonUtils.pyObjectToDatum(result); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean equals(Object o) { + if (o instanceof PythonFunctionInvoke) { + PythonFunctionInvoke other = (PythonFunctionInvoke) o; + return this.invokeDesc.equals(other.invokeDesc) && + TUtil.checkEquals(this.paramTypes, other.paramTypes); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(invokeDesc, Arrays.hashCode(paramTypes)); + } + + @Override + public Object clone() throws CloneNotSupportedException { + PythonFunctionInvoke clone = (PythonFunctionInvoke) super.clone(); + clone.invokeDesc = (PythonInvocationDesc) this.invokeDesc.clone(); + clone.paramTypes = new TajoDataTypes.DataType[paramTypes.length]; + paramTypes = Arrays.copyOf(paramTypes, paramTypes.length); + return clone; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java new file mode 100644 index 0000000000..b11dd09c88 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java @@ -0,0 +1,201 @@ +package org.apache.tajo.plan.expr; + +import com.google.common.base.Charsets; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.plan.function.python.ScriptingOutputCapturer; +import org.apache.tajo.plan.function.stream.StreamingUDFException; +import org.apache.tajo.storage.Tuple; + +import java.io.*; +import java.util.concurrent.BlockingQueue; + +public class PythonFunctionInvoke2 extends FunctionInvoke { + + private static final Log log = LogFactory.getLog(PythonFunctionInvoke2.class); + + private static final String PYTHON_CONTROLLER_JAR_PATH = "/python/controller.py"; //Relative to root of tajo jar. + private static final String PYTHON_PIG_UTIL_PATH = "/python/tajo_util.py"; //Relative to root of tajo jar. + + //Indexes for arguments being passed to external process + private static final int UDF_LANGUAGE = 0; + private static final int PATH_TO_CONTROLLER_FILE = 1; + private static final int UDF_FILE_NAME = 2; //Name of file where UDF function is defined + private static final int UDF_FILE_PATH = 3; //Path to directory containing file where UDF function is defined + private static final int UDF_NAME = 4; //Name of UDF function being called. + private static final int PATH_TO_FILE_CACHE = 5; //Directory where required files (like tajo_util) are cached on cluster nodes. + private static final int STD_OUT_OUTPUT_PATH = 6; //File for output from when user writes to standard output. + private static final int STD_ERR_OUTPUT_PATH = 7; //File for output from when user writes to standard error. + private static final int CONTROLLER_LOG_FILE_PATH = 8; //Controller log file logs progress through the controller script not user code. + private static final int IS_ILLUSTRATE = 9; //Controller captures output differently in illustrate vs running. + + private String language; + private String filePath; + private String funcName; + + private ScriptingOutputCapturer soc; + + private Process process; // Handle to the externwlgns1441 + // al process + private ProcessErrorThread stderrThread; // thread to get process stderr + private ProcessInputThread stdinThread; // thread to send input to process + private ProcessOutputThread stdoutThread; //thread to read output from process + + private InputHandler inputHandler; + private OutputHandler outputHandler; + + private BlockingQueue inputQueue; + private BlockingQueue outputQueue; + + private DataOutputStream stdin; // stdin of the process + private InputStream stdout; // stdout of the process + private InputStream stderr; // stderr of the process + + private static final Object ERROR_OUTPUT = new Object(); + private static final Object NULL_OBJECT = new Object(); //BlockingQueue can't have null. Use place holder object instead. + + private volatile StreamingUDFException outerrThreadsError; + + public static final String TURN_ON_OUTPUT_CAPTURING = "TURN_ON_OUTPUT_CAPTURING"; + + public PythonFunctionInvoke2(FunctionDesc functionDesc) { + super(functionDesc); + } + + @Override + public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { + + } + + @Override + public Datum eval(Tuple tuple) { + return null; + } + + /** + * The thread which consumes input and feeds it to the the Process + */ + class ProcessInputThread extends Thread { + ProcessInputThread() { + setDaemon(true); + } + + public void run() { + try { + log.debug("Starting PIT"); + while (true) { + Tuple inputTuple = inputQueue.take(); + inputHandler.putNext(inputTuple); + try { + stdin.flush(); + } catch(Exception e) { + return; + } + } + } catch (Exception e) { + log.error(e); + } + } + } + + private static final int WAIT_FOR_ERROR_LENGTH = 500; + private static final int MAX_WAIT_FOR_ERROR_ATTEMPTS = 5; + + /** + * The thread which consumes output from process + */ + class ProcessOutputThread extends Thread { + ProcessOutputThread() { + setDaemon(true); + } + + public void run() { + Object o = null; + try{ + log.debug("Starting POT"); + //StreamUDFToPig wraps object in single element tuple + o = outputHandler.getNext().get(0); + while (o != OutputHandler.END_OF_OUTPUT) { + if (o != null) + outputQueue.put(o); + else + outputQueue.put(NULL_OBJECT); + o = outputHandler.getNext().get(0); + } + } catch(Exception e) { + if (outputQueue != null) { + try { + //Give error thread a chance to check the standard error output + //for an exception message. + int attempt = 0; + while (stderrThread.isAlive() && attempt < MAX_WAIT_FOR_ERROR_ATTEMPTS) { + Thread.sleep(WAIT_FOR_ERROR_LENGTH); + attempt++; + } + //Only write this if no other error. Don't want to overwrite + //an error from the error thread. + if (outerrThreadsError == null) { + outerrThreadsError = new StreamingUDFException( + language, "Error deserializing output. Please check that the declared outputSchema for function " + + funcName + " matches the data type being returned.", e); + } + outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. + } catch(InterruptedException ie) { + log.error(ie); + } + } + } + } + } + + class ProcessErrorThread extends Thread { + public ProcessErrorThread() { + setDaemon(true); + } + + public void run() { + try { + log.debug("Starting PET"); + Integer lineNumber = null; + StringBuffer error = new StringBuffer(); + String errInput; + BufferedReader reader = new BufferedReader( + new InputStreamReader(stderr, Charsets.UTF_8)); + while ((errInput = reader.readLine()) != null) { + //First line of error stream is usually the line number of error. + //If its not a number just treat it as first line of error message. + if (lineNumber == null) { + try { + lineNumber = Integer.valueOf(errInput); + } catch (NumberFormatException nfe) { + error.append(errInput + "\n"); + } + } else { + error.append(errInput + "\n"); + } + } + outerrThreadsError = new StreamingUDFException(language, error.toString(), lineNumber); + if (outputQueue != null) { + outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. + } + if (stderr != null) { + stderr.close(); + stderr = null; + } + } catch (IOException e) { + log.debug("Process Ended", e); + } catch (Exception e) { + log.error("standard error problem", e); + } + } + } + + public class ProcessKiller implements Runnable { + public void run() { + process.destroy(); + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java index d0577587e5..e706391c38 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/SimpleEvalNodeVisitor.java @@ -74,9 +74,6 @@ public EvalNode visit(CONTEXT context, EvalNode evalNode, Stack stack) case WINDOW_FUNCTION: result = visitFuncCall(context, (FunctionEval) evalNode, stack); break; - case PYTHON_FUNCTION: - result = visitFuncCall(context, (FunctionEval) evalNode, stack); - break; default: throw new UnsupportedException("Unknown EvalType: " + evalNode); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index c307d8260a..6940b963d3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -279,7 +279,7 @@ public static Set registerFunctions(String path, String namespace) } // Parameters have the ANY type. - int paramNum = ((PyBaseCode) pyFunction.func_code).co_argcount; + int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; for (int i = 0; i < paramNum; i++) { paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java new file mode 100644 index 0000000000..bf62fd0275 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java @@ -0,0 +1,151 @@ +package org.apache.tajo.plan.function.python; + +import com.google.common.base.Charsets; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.util.TUtil; + +import java.io.*; +import java.util.Map; +import java.util.UUID; + +public class ScriptingOutputCapturer { + private static Log log = LogFactory.getLog(ScriptingOutputCapturer.class); + + private static Map outputFileNames = TUtil.newHashMap(); + //Unique ID for this run to ensure udf output files aren't corrupted from previous runs. + private static String runId = UUID.randomUUID().toString(); + + //Illustrate will set the static flag telling udf to start capturing its output. It's up to each + //instance to react to it and set its own flag. + private static boolean captureOutput = false; + private boolean instancedCapturingOutput = false; + + private FunctionDesc functionDesc; + private OverridableConf queryContext; + + public ScriptingOutputCapturer(OverridableConf queryContext, FunctionDesc functionDesc) { + this.queryContext = queryContext; + this.functionDesc = functionDesc; + } + + public String getStandardOutputRootWriteLocation() throws IOException { +// Configuration conf = UDFContext.getUDFContext().getJobConf(); +// +// String jobId = conf.get(MRConfiguration.JOB_ID); +// String taskId = conf.get(MRConfiguration.TASK_ID); +// String hadoopLogDir = System.getProperty("yarn.app.container.log.dir"); +// if (hadoopLogDir == null) { +// hadoopLogDir = conf.get("yarn.app.container.log.dir"); +// } +// if (hadoopLogDir == null) { +// hadoopLogDir = System.getProperty("hadoop.log.dir"); +// } +// if (hadoopLogDir == null) { +// hadoopLogDir = conf.get("hadoop.log.dir"); +// } +// +// String tmpDir = conf.get("hadoop.tmp.dir"); +// boolean fallbackToTmp = (hadoopLogDir == null); +// if (!fallbackToTmp) { +// try { +// if (!(new File(hadoopLogDir).canWrite())) { +// fallbackToTmp = true; +// } +// } +// catch (SecurityException e) { +// fallbackToTmp = true; +// } +// finally { +// if (fallbackToTmp) +// log.warn(String.format("Insufficient permission to write into %s. Change path to: %s", hadoopLogDir, tmpDir)); +// } +// } +// if (fallbackToTmp) { +// hadoopLogDir = tmpDir; +// } +// log.debug("JobId: " + jobId); +// log.debug("TaskId: " + taskId); +// log.debug("hadoopLogDir: " + hadoopLogDir); +// +// if (execType.isLocal() || conf.getBoolean(PigImplConstants.CONVERTED_TO_FETCH, false)) { +// String logDir = System.getProperty("pig.udf.scripting.log.dir"); +// if (logDir == null) +// logDir = "."; +// return logDir + "/" + (taskId == null ? "" : (taskId + "_")); +// } else { +// String taskLogDir = getTaskLogDir(jobId, taskId, hadoopLogDir); +// return taskLogDir + "/"; +// } + return null; + } + + public String getTaskLogDir(String jobId, String taskId, String hadoopLogDir) throws IOException { + String taskLogDir = null; +// String defaultUserLogDir = hadoopLogDir + File.separator + "userlogs"; +// +// if ( new File(defaultUserLogDir + File.separator + jobId).exists() ) { +// taskLogDir = defaultUserLogDir + File.separator + jobId + File.separator + taskId; +// } else if ( new File(defaultUserLogDir + File.separator + taskId).exists() ) { +// taskLogDir = defaultUserLogDir + File.separator + taskId; +// } else if ( new File(defaultUserLogDir).exists() ){ +// taskLogDir = defaultUserLogDir; +// } else { +// taskLogDir = hadoopLogDir + File.separator + "udfOutput"; +// File dir = new File(taskLogDir); +// dir.mkdirs(); +// if (!dir.exists()) { +// throw new IOException("Could not create directory: " + taskLogDir); +// } +// } + return taskLogDir; + } + + public static void startCapturingOutput() { + ScriptingOutputCapturer.captureOutput = true; + } + + public static Map getUdfOutput() throws IOException { + Map udfFuncNameToOutput = TUtil.newHashMap(); + for (Map.Entry funcToOutputFileName : outputFileNames.entrySet()) { + StringBuffer udfOutput = new StringBuffer(); + FileInputStream fis = new FileInputStream(funcToOutputFileName.getValue()); + Reader fr = new InputStreamReader(fis, Charsets.UTF_8); + BufferedReader br = new BufferedReader(fr); + + try { + String line = br.readLine(); + while (line != null) { + udfOutput.append("\t" + line + "\n"); + line = br.readLine(); + } + } finally { + br.close(); + } + udfFuncNameToOutput.put(funcToOutputFileName.getKey(), udfOutput.toString()); + } + return udfFuncNameToOutput; + } + + public void registerOutputLocation(String functionName, String fileName) { + outputFileNames.put(functionName, fileName); + } + + public static String getRunId() { + return runId; + } + + public static boolean isClassCapturingOutput() { + return ScriptingOutputCapturer.captureOutput; + } + + public boolean isInstanceCapturingOutput() { + return this.instancedCapturingOutput; + } + + public void setInstanceCapturingOutput(boolean instanceCapturingOutput) { + this.instancedCapturingOutput = instanceCapturingOutput; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java new file mode 100644 index 0000000000..4bb6b3d25b --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -0,0 +1,86 @@ +package org.apache.tajo.plan.function.stream; + +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; +import java.io.OutputStream; + +/** + * {@link InputHandler} is responsible for handling the input to the + * Pig-Streaming external command. + * + * The managed executable could be fed input in a {@link InputType#SYNCHRONOUS} + * manner via its stdin or in an {@link InputType#ASYNCHRONOUS} + * manner via an external file which is subsequently read by the executable. + */ +public abstract class InputHandler { + /** + * + */ + public enum InputType {SYNCHRONOUS, ASYNCHRONOUS} + + /** + * The serializer to be used to send data to the managed process. + * + * It is the responsibility of the concrete sub-classes to setup and + * manage the serializer. + */ + protected RowStoreUtil.RowStoreEncoder serializer; + + private OutputStream out; + + // flag to mark if close() has already been called + protected boolean alreadyClosed = false; + + /** + * Get the handled InputType + * @return the handled InputType + */ + public abstract InputType getInputType(); + + /** + * Send the given input Tuple to the managed executable. + * + * @param t input Tuple + * @throws IOException + */ + public void putNext(Tuple t) throws IOException { + out.write(serializer.toBytes(t)); + } + + /** + * Close the InputHandler since there is no more input + * to be sent to the managed process. + * @param process the managed process - this could be null in some cases + * like when input is through files. In that case, the process would not + * have been exec'ed yet - if this method if overridden it is the responsibility + * of the implementer to check that the process is usable. The managed process + * object is supplied by the ExecutableManager to this call so that this method + * can check if the process is alive if it needs to know. + * + * @throws IOException + */ + public synchronized void close(Process process) throws IOException { + if(!alreadyClosed) { + alreadyClosed = true; + out.flush(); + out.close(); + out = null; + } + } + + /** + * Bind the InputHandler to the OutputStream + * from which it reads input and sends it to the managed process. + * + * @param os OutputStream from which to read input data for the + * managed process + * @throws IOException + */ + public void bindTo(OutputStream os) throws IOException { + out = os; + if (this.serializer instanceof PigStreamingBase) { + this.newSerializer = (PigStreamingBase) serializer; + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java new file mode 100644 index 0000000000..0d71a51ccc --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java @@ -0,0 +1,272 @@ +package org.apache.tajo.plan.function.stream; + +import org.apache.tajo.catalog.Column; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.DatumFactory; +import org.apache.tajo.datum.IntervalDatum; +import org.apache.tajo.exception.UnknownDataTypeException; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; +import org.apache.tajo.util.BitArray; + +import java.nio.ByteBuffer; + +public class RowStoreUtil { + + public static int[] getTargetIds(Schema inSchema, Schema outSchema) { + int[] targetIds = new int[outSchema.size()]; + int i = 0; + for (Column target : outSchema.getColumns()) { + targetIds[i] = inSchema.getColumnId(target.getQualifiedName()); + i++; + } + + return targetIds; + } + + public static Tuple project(Tuple in, Tuple out, int[] targetIds) { + out.clear(); + for (int idx = 0; idx < targetIds.length; idx++) { + out.put(idx, in.get(targetIds[idx])); + } + return out; + } + + public static RowStoreEncoder createEncoder(Schema schema) { + return new RowStoreEncoder(schema); + } + + public static RowStoreDecoder createDecoder(Schema schema) { + return new RowStoreDecoder(schema); + } + + public static class RowStoreDecoder { + + private Schema schema; + private BitArray nullFlags; + private int headerSize; + + private RowStoreDecoder(Schema schema) { + this.schema = schema; + nullFlags = new BitArray(schema.size()); + headerSize = nullFlags.bytesLength(); + } + + + public Tuple toTuple(byte [] bytes) { + nullFlags.clear(); + ByteBuffer bb = ByteBuffer.wrap(bytes); + Tuple tuple = new VTuple(schema.size()); + Column col; + TajoDataTypes.DataType type; + + bb.limit(headerSize); + nullFlags.fromByteBuffer(bb); + bb.limit(bytes.length); + + for (int i =0; i < schema.size(); i++) { + if (nullFlags.get(i)) { + tuple.put(i, DatumFactory.createNullDatum()); + continue; + } + + col = schema.getColumn(i); + type = col.getDataType(); + switch (type.getType()) { + case BOOLEAN: tuple.put(i, DatumFactory.createBool(bb.get())); break; + case BIT: + byte b = bb.get(); + tuple.put(i, DatumFactory.createBit(b)); + break; + + case CHAR: + byte c = bb.get(); + tuple.put(i, DatumFactory.createChar(c)); + break; + + case INT2: + short s = bb.getShort(); + tuple.put(i, DatumFactory.createInt2(s)); + break; + + case INT4: + case DATE: + int i_ = bb.getInt(); + tuple.put(i, DatumFactory.createFromInt4(type, i_)); + break; + + case INT8: + case TIME: + case TIMESTAMP: + long l = bb.getLong(); + tuple.put(i, DatumFactory.createFromInt8(type, l)); + break; + + case INTERVAL: + int month = bb.getInt(); + long milliseconds = bb.getLong(); + tuple.put(i, new IntervalDatum(month, milliseconds)); + break; + + case FLOAT4: + float f = bb.getFloat(); + tuple.put(i, DatumFactory.createFloat4(f)); + break; + + case FLOAT8: + double d = bb.getDouble(); + tuple.put(i, DatumFactory.createFloat8(d)); + break; + + case TEXT: + byte [] _string = new byte[bb.getInt()]; + bb.get(_string); + tuple.put(i, DatumFactory.createText(_string)); + break; + + case BLOB: + byte [] _bytes = new byte[bb.getInt()]; + bb.get(_bytes); + tuple.put(i, DatumFactory.createBlob(_bytes)); + break; + + case INET4: + byte [] _ipv4 = new byte[4]; + bb.get(_ipv4); + tuple.put(i, DatumFactory.createInet4(_ipv4)); + break; + case INET6: + // TODO - to be implemented + throw new UnsupportedException(type.getType().name()); + default: + throw new RuntimeException(new UnknownDataTypeException(type.getType().name())); + } + } + return tuple; + } + + public Schema getSchema() { + return schema; + } + } + + public static class RowStoreEncoder { + private Schema schema; + private BitArray nullFlags; + private int headerSize; + + private RowStoreEncoder(Schema schema) { + this.schema = schema; + nullFlags = new BitArray(schema.size()); + headerSize = nullFlags.bytesLength(); + } + public byte [] toBytes(Tuple tuple) { + nullFlags.clear(); + int size = estimateTupleDataSize(tuple); + ByteBuffer bb = ByteBuffer.allocate(size + headerSize); + bb.position(headerSize); + Column col; + for (int i = 0; i < schema.size(); i++) { + if (tuple.isNull(i)) { + nullFlags.set(i); + continue; + } + + col = schema.getColumn(i); + switch (col.getDataType().getType()) { + case NULL_TYPE: nullFlags.set(i); break; + case BOOLEAN: bb.put(tuple.get(i).asByte()); break; + case BIT: bb.put(tuple.get(i).asByte()); break; + case CHAR: bb.put(tuple.get(i).asByte()); break; + case INT2: bb.putShort(tuple.get(i).asInt2()); break; + case INT4: bb.putInt(tuple.get(i).asInt4()); break; + case INT8: bb.putLong(tuple.get(i).asInt8()); break; + case FLOAT4: bb.putFloat(tuple.get(i).asFloat4()); break; + case FLOAT8: bb.putDouble(tuple.get(i).asFloat8()); break; + case TEXT: + byte [] _string = tuple.get(i).asByteArray(); + bb.putInt(_string.length); + bb.put(_string); + break; + case DATE: bb.putInt(tuple.get(i).asInt4()); break; + case TIME: + case TIMESTAMP: + bb.putLong(tuple.get(i).asInt8()); + break; + case INTERVAL: + IntervalDatum interval = (IntervalDatum) tuple.get(i); + bb.putInt(interval.getMonths()); + bb.putLong(interval.getMilliSeconds()); + break; + case BLOB: + byte [] bytes = tuple.get(i).asByteArray(); + bb.putInt(bytes.length); + bb.put(bytes); + break; + case INET4: + byte [] ipBytes = tuple.get(i).asByteArray(); + bb.put(ipBytes); + break; + case INET6: bb.put(tuple.get(i).asByteArray()); break; + default: + throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name())); + } + } + + byte[] flags = nullFlags.toArray(); + int finalPosition = bb.position(); + bb.position(0); + bb.put(flags); + + bb.position(finalPosition); + bb.flip(); + byte [] buf = new byte [bb.limit()]; + bb.get(buf); + return buf; + } + + // Note that, NULL values are treated separately + private int estimateTupleDataSize(Tuple tuple) { + int size = 0; + Column col; + + for (int i = 0; i < schema.size(); i++) { + if (tuple.isNull(i)) { + continue; + } + + col = schema.getColumn(i); + switch (col.getDataType().getType()) { + case BOOLEAN: + case BIT: + case CHAR: size += 1; break; + case INT2: size += 2; break; + case DATE: + case INT4: + case FLOAT4: size += 4; break; + case TIME: + case TIMESTAMP: + case INT8: + case FLOAT8: size += 8; break; + case INTERVAL: size += 12; break; + case TEXT: + case BLOB: size += (4 + tuple.get(i).asByteArray().length); break; + case INET4: + case INET6: size += tuple.get(i).asByteArray().length; break; + default: + throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name())); + } + } + + size += 100; // optimistic reservation + + return size; + } + + public Schema getSchema() { + return schema; + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java new file mode 100644 index 0000000000..706e748cfd --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java @@ -0,0 +1,57 @@ +package org.apache.tajo.plan.function.stream; + +public class StreamingUDFException extends Exception { + + private String message; + private String language; + private Integer lineNumber; + + public StreamingUDFException() { + } + + public StreamingUDFException(String message) { + this.message = message; + } + + public StreamingUDFException(String message, Integer lineNumber) { + this.message = message; + this.lineNumber = lineNumber; + } + + public StreamingUDFException(String language, String message, Throwable cause) { + super(cause); + this.language = language; + this.message = message + "\n" + cause.getMessage() + "\n"; + } + + public StreamingUDFException(String language, String message) { + this(language, message, (Integer) null); + } + + public StreamingUDFException(String language, String message, Integer lineNumber) { + this.language = language; + this.message = message; + this.lineNumber = lineNumber; + } + + public String getLanguage() { + return language; + } + + public Integer getLineNumber() { + return lineNumber; + } + + @Override + public String getMessage() { + return this.message; + } + + @Override + public String toString() { + String s = getClass().getName(); + String message = getMessage(); + String lineNumber = this.getLineNumber() == null ? "" : "" + this.getLineNumber(); + return (message != null) ? (s + ": " + "LINE " + lineNumber + ": " + message) : s; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 892b2941af..caa454517b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -180,9 +180,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { try { funcDesc = new FunctionDesc(funcProto.getFuncion()); if (type == EvalType.FUNCTION) { - GeneralFunction instance = (GeneralFunction) funcDesc.newInstance(); - current = new GeneralFunctionEval(context, new FunctionDesc(funcProto.getFuncion()), instance, params); - + current = new GeneralFunctionEval(context, new FunctionDesc(funcProto.getFuncion()), params); } else if (type == EvalType.AGG_FUNCTION || type == EvalType.WINDOW_FUNCTION) { AggFunction instance = (AggFunction) funcDesc.newInstance(); if (type == EvalType.AGG_FUNCTION) { @@ -211,8 +209,6 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { current = winFunc; } - } else if (type == EvalType.PYTHON_FUNCTION) { - current = new GeneralPythonFunctionEval(new FunctionDesc(funcProto.getFuncion()), params); } } catch (ClassNotFoundException cnfe) { String functionName = "Unknown"; From 1d4acb3338bddec81c2a2d8232af202f3695ae03 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 3 Apr 2015 22:11:14 +0900 Subject: [PATCH 23/55] TAJO-1344 --- .../function/stream/DefaultInputHandler.java | 36 +++++++++++++++++++ .../plan/function/stream/InputHandler.java | 3 -- .../plan/function/stream/StringCommand.java | 4 +++ 3 files changed, 40 insertions(+), 3 deletions(-) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java new file mode 100644 index 0000000000..6f3aa73459 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java @@ -0,0 +1,36 @@ +package org.apache.tajo.plan.function.stream; + +import org.apache.tajo.catalog.Schema; + +import java.io.IOException; + +public class DefaultInputHandler extends InputHandler { + + public DefaultInputHandler(Schema schema) { + serializer = RowStoreUtil.createEncoder(schema); + } + + public DefaultInputHandler(HandleSpec spec) { + serializer = (PigToStream)PigContext.instantiateFuncFromSpec(spec.spec); + } + + @Override + public InputType getInputType() { + return InputType.SYNCHRONOUS; + } + + @Override + public synchronized void close(Process process) throws IOException { + try { + super.close(process); + } catch(IOException e) { + // check if we got an exception because + // the process actually completed and we were + // trying to flush and close it's stdin + if (process == null || process.exitValue() != 0) { + // the process had not terminated normally + // throw the exception we got + throw e; + } + } + } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java index 4bb6b3d25b..c4a2b3d525 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -79,8 +79,5 @@ public synchronized void close(Process process) throws IOException { */ public void bindTo(OutputStream os) throws IOException { out = os; - if (this.serializer instanceof PigStreamingBase) { - this.newSerializer = (PigStreamingBase) serializer; - } } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java new file mode 100644 index 0000000000..12775b860d --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java @@ -0,0 +1,4 @@ +package org.apache.tajo.plan.function.stream; + +public class StringCommand { +} From 6cc6f39e0b113c82328b3e6693ad1b0af933abb3 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sat, 4 Apr 2015 18:32:34 +0900 Subject: [PATCH 24/55] TAJO-1344 --- .../engine/function/TestPythonFunctions.java | 12 +- .../org/apache/tajo/plan/ExprAnnotator.java | 3 + .../apache/tajo/plan/expr/FunctionInvoke.java | 15 +- .../tajo/plan/expr/GeneralFunctionEval.java | 4 +- .../tajo/plan/expr/PythonFunctionInvoke.java | 1 + .../tajo/plan/expr/PythonFunctionInvoke2.java | 299 ++++++++- .../apache/tajo/plan/expr/StreamingUtil.java | 125 ++++ .../function/python/JythonScriptEngine.java | 624 +++++++++--------- .../plan/function/python/JythonUtils.java | 43 ++ .../python/ScriptingOutputCapturer.java | 35 +- .../function/python/TajoScriptEngine.java | 192 +++--- .../tajo/plan/function/stream/BufferPool.java | 74 +++ .../function/stream/ByteBufInputChannel.java | 71 ++ .../function/stream/ByteBufLineReader.java | 180 +++++ .../function/stream/DefaultInputHandler.java | 36 - .../plan/function/stream/InputHandler.java | 18 + .../function/stream/LineSplitProcessor.java | 45 ++ .../plan/function/stream/OutputHandler.java | 161 +++++ .../plan/function/stream/RowStoreUtil.java | 18 + .../function/stream/StreamingCommand.java | 477 +++++++++++++ .../stream/StreamingUDFException.java | 18 + .../stream/StreamingUDFInputHandler.java | 49 ++ .../stream/StreamingUDFOutputHandler.java | 36 + .../plan/function/stream/StringCommand.java | 4 - .../plan/serder/EvalNodeDeserializer.java | 3 + 25 files changed, 2067 insertions(+), 476 deletions(-) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/LineSplitProcessor.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java index 555b2cede2..869ffb65fa 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -27,12 +27,12 @@ public class TestPythonFunctions extends ExprTestBase { @Test public void test() throws IOException { - testSimpleEval("select sum_py(1,2)", new String[]{"3"}); testSimpleEval("select return_one()", new String[]{"1"}); - testSimpleEval("select helloworld()", new String[]{"Hello, World"}); - testSimpleEval("select concat_py('1')", new String[]{"11"}); - testSimpleEval("select comma_format(12345)", new String[]{"12,345"}); - testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); - testSimpleEval("select percent(386, 1000)", new String[]{"38.6"}); +// testSimpleEval("select helloworld()", new String[]{"Hello, World"}); +// testSimpleEval("select sum_py(1,2)", new String[]{"3"}); +// testSimpleEval("select concat_py('1')", new String[]{"11"}); +// testSimpleEval("select comma_format(12345)", new String[]{"12,345"}); +// testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); +// testSimpleEval("select percent(386, 1000)", new String[]{"38.6"}); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index b5d883136b..d2a2138e07 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -43,6 +43,7 @@ import org.apache.tajo.util.datetime.DateTimeUtil; import org.apache.tajo.util.datetime.TimeMeta; +import java.io.IOException; import java.util.Set; import java.util.Stack; import java.util.TimeZone; @@ -623,6 +624,8 @@ public EvalNode visitFunction(Context ctx, Stack stack, FunctionExpr expr) } } catch (InternalException e) { throw new PlanningException(e); + } catch (IOException e) { + throw new PlanningException(e); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java index 66ce6293b2..24e77a514b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java @@ -21,11 +21,14 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.OverridableConf; import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; import org.apache.tajo.exception.InternalException; import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.storage.Tuple; +import java.io.IOException; + public abstract class FunctionInvoke { @Expose protected FunctionDesc functionDesc; @@ -37,13 +40,21 @@ public static FunctionInvoke newInstance(FunctionDesc desc) throws InternalExcep if (desc.getInvocation().hasLegacy()) { return new LegacyScalarFunctionInvoke(desc); } else if (desc.getInvocation().hasPython()) { - return new PythonFunctionInvoke(desc); +// return new PythonFunctionInvoke(desc); + return new PythonFunctionInvoke2(desc); } else { throw new UnsupportedException(desc.getInvocation() + " is not supported"); } } - public abstract void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes); + public abstract void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) throws IOException; + + +// /** +// * Bind an input schema. +// * @param schema intpu schema +// */ +// public abstract void bind(Schema schema); /** * Evaluate the given tuple with a function diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 462ddc81fd..25cedc9c4a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -30,11 +30,13 @@ import org.apache.tajo.util.TUtil; import javax.annotation.Nullable; +import java.io.IOException; public class GeneralFunctionEval extends FunctionEval { @Expose protected FunctionInvoke funcInvoke; - public GeneralFunctionEval(@Nullable OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) throws InternalException { + public GeneralFunctionEval(@Nullable OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) + throws IOException { super(EvalType.FUNCTION, desc, givenArgs); this.funcInvoke = FunctionInvoke.newInstance(desc); this.funcInvoke.init(queryContext, getParamType()); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java index e083824023..3145a7770a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java @@ -22,6 +22,7 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.OverridableConf; import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; import org.apache.tajo.function.PythonInvocationDesc; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java index b11dd09c88..c790ebabf6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java @@ -1,16 +1,45 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.tajo.plan.expr; import com.google.common.base.Charsets; +import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.DatumFactory; +import org.apache.tajo.function.FunctionInvocation; +import org.apache.tajo.function.FunctionSignature; +import org.apache.tajo.function.PythonInvocationDesc; +import org.apache.tajo.plan.function.python.JythonUtils; import org.apache.tajo.plan.function.python.ScriptingOutputCapturer; -import org.apache.tajo.plan.function.stream.StreamingUDFException; +import org.apache.tajo.plan.function.stream.*; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; import java.io.*; +import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; public class PythonFunctionInvoke2 extends FunctionInvoke { @@ -32,10 +61,6 @@ public class PythonFunctionInvoke2 extends FunctionInvoke { private static final int CONTROLLER_LOG_FILE_PATH = 8; //Controller log file logs progress through the controller script not user code. private static final int IS_ILLUSTRATE = 9; //Controller captures output differently in illustrate vs running. - private String language; - private String filePath; - private String funcName; - private ScriptingOutputCapturer soc; private Process process; // Handle to the externwlgns1441 @@ -59,20 +84,274 @@ public class PythonFunctionInvoke2 extends FunctionInvoke { private volatile StreamingUDFException outerrThreadsError; + private OverridableConf queryContext = null; + + private FunctionSignature functionSignature; + private PythonInvocationDesc invocationDesc; + private Schema inSchema; + private Schema outSchema; + private boolean isBinded = false; + public static final String TURN_ON_OUTPUT_CAPTURING = "TURN_ON_OUTPUT_CAPTURING"; public PythonFunctionInvoke2(FunctionDesc functionDesc) { super(functionDesc); + if (!functionDesc.getInvocation().hasPython()) { + throw new IllegalStateException("Function type must be python"); + } + functionSignature = functionDesc.getSignature(); + invocationDesc = functionDesc.getInvocation().getPython(); + + // Compile input/output schema + // Note that temporal columns are used. + TajoDataTypes.DataType[] paramTypes = functionSignature.getParamTypes(); + inSchema = new Schema(); + for (int i = 0; i < paramTypes.length; i++) { + inSchema.addColumn(new Column("in_" + i, paramTypes[i])); + } + outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); } @Override - public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { + public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) throws IOException { + this.queryContext = queryContext; + this.inputQueue = new ArrayBlockingQueue(1); + this.outputQueue = new ArrayBlockingQueue(2); + this.soc = new ScriptingOutputCapturer(queryContext, functionDesc); + startUdfController(); + createInputHandlers(); + setStreams(); + startThreads(); + } + + private StreamingCommand startUdfController() throws IOException { + StreamingCommand sc = new StreamingCommand(constructCommand()); + ProcessBuilder processBuilder = StreamingUtil.createProcess(queryContext, sc); + process = processBuilder.start(); + + Runtime.getRuntime().addShutdownHook(new Thread(new ProcessKiller())); + return sc; + } + + private String[] constructCommand() throws IOException { + String[] command = new String[10]; + +// String jarPath = conf.get("mapreduce.job.jar"); +// if (jarPath == null) { +// jarPath = conf.get(MRConfiguration.JAR); +// } +// String jobDir; +// if (jarPath != null) { +// jobDir = new File(jarPath).getParent(); +// } else { +// jobDir = ""; +// } + +// String standardOutputRootWriteLocation = soc.getStandardOutputRootWriteLocation(); + String standardOutputRootWriteLocation = System.getProperty("TAJO_LOG_DIR"); + String controllerLogFileName, outFileName, errOutFileName; + +// if (execType.isLocal()) { +// controllerLogFileName = standardOutputRootWriteLocation + funcName + "_python.log"; +// outFileName = standardOutputRootWriteLocation + "cpython_" + funcName + "_" + ScriptingOutputCapturer.getRunId() + ".out"; +// errOutFileName = standardOutputRootWriteLocation + "cpython_" + funcName + "_" + ScriptingOutputCapturer.getRunId() + ".err"; +// } else { + String funcName = invocationDesc.getName(); + String filePath = invocationDesc.getPath(); + + controllerLogFileName = standardOutputRootWriteLocation + funcName + "_python.log"; + outFileName = standardOutputRootWriteLocation + funcName + ".out"; + errOutFileName = standardOutputRootWriteLocation + funcName + ".err"; +// } + + soc.registerOutputLocation(funcName, outFileName); + + command[UDF_LANGUAGE] = "python"; + command[PATH_TO_CONTROLLER_FILE] = getControllerPath(); + int lastSeparator = filePath.lastIndexOf(File.separator) + 1; + command[UDF_FILE_NAME] = filePath.substring(lastSeparator); + command[UDF_FILE_PATH] = lastSeparator <= 0 ? + "." : + filePath.substring(0, lastSeparator - 1); + command[UDF_NAME] = funcName; + String fileCachePath = filePath.substring(0, lastSeparator); + command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; + command[STD_OUT_OUTPUT_PATH] = outFileName; + command[STD_ERR_OUTPUT_PATH] = errOutFileName; + command[CONTROLLER_LOG_FILE_PATH] = controllerLogFileName; + command[IS_ILLUSTRATE] = ""; + +// ensureUserFileAvailable(command, fileCachePath); + + return command; + } + + /** + * Need to make sure the user's file is available. If jar hasn't been + * exploded, just copy the udf file to its path relative to the controller + * file and update file cache path appropriately. + */ + private void ensureUserFileAvailable(String[] command, String fileCachePath) + throws IOException { + + File userUdfFile = new File(fileCachePath + command[UDF_FILE_NAME]); + if (!userUdfFile.exists()) { + String filePath = invocationDesc.getPath(); + String absolutePath = filePath.startsWith("/") ? filePath : "/" + filePath; + absolutePath = absolutePath.replaceAll(":", ""); + String controllerDir = new File(command[PATH_TO_CONTROLLER_FILE]).getParent(); + String userUdfPath = controllerDir + absolutePath + getUserFileExtension(); + userUdfFile = new File(userUdfPath); + userUdfFile.deleteOnExit(); + userUdfFile.getParentFile().mkdirs(); + if (userUdfFile.exists()) { + userUdfFile.delete(); + if (!userUdfFile.createNewFile()) { + throw new IOException("Unable to create file: " + userUdfFile.getAbsolutePath()); + } + } + InputStream udfFileStream = this.getClass().getResourceAsStream( + absolutePath + getUserFileExtension()); + command[PATH_TO_FILE_CACHE] = "\"" + userUdfFile.getParentFile().getAbsolutePath() + + "\""; + + try { + FileUtils.copyInputStreamToFile(udfFileStream, userUdfFile); + } + catch (Exception e) { + throw new IOException("Unable to copy user udf file: " + userUdfFile.getName(), e); + } + finally { + udfFileStream.close(); + } + } + } + private String getUserFileExtension() { + return ".py"; + } + + private void createInputHandlers() { + RowStoreUtil.RowStoreEncoder serializer = RowStoreUtil.createEncoder(inSchema); + this.inputHandler = new StreamingUDFInputHandler(serializer); + RowStoreUtil.RowStoreDecoder deserializer = RowStoreUtil.createDecoder(outSchema); + this.outputHandler = new StreamingUDFOutputHandler(deserializer); + } + + private void setStreams() throws IOException { + stdout = new DataInputStream(new BufferedInputStream(process + .getInputStream())); + outputHandler.bindTo("", stdout, + 0, Long.MAX_VALUE); + + stdin = new DataOutputStream(new BufferedOutputStream(process + .getOutputStream())); + inputHandler.bindTo(stdin); + + stderr = new DataInputStream(new BufferedInputStream(process + .getErrorStream())); + } + + private void startThreads() { + stdinThread = new ProcessInputThread(); + stdinThread.start(); + + stdoutThread = new ProcessOutputThread(); + stdoutThread.start(); + + stderrThread = new ProcessErrorThread(); + stderrThread.start(); + } + + /** + * Find the path to the controller file for the streaming language. + * + * First check path to job jar and if the file is not found (like in the + * case of running hadoop in standalone mode) write the necessary files + * to temporary files and return that path. + * + * @return + * @throws IOException + */ + private String getControllerPath() throws IOException { + String controllerPath = PYTHON_CONTROLLER_JAR_PATH; + File controller = new File(PYTHON_CONTROLLER_JAR_PATH); + if (!controller.exists()) { + File controllerFile = File.createTempFile("controller", ".py"); + InputStream pythonControllerStream = this.getClass().getResourceAsStream(PYTHON_CONTROLLER_JAR_PATH); + try { + FileUtils.copyInputStreamToFile(pythonControllerStream, controllerFile); + } finally { + pythonControllerStream.close(); + } + controllerFile.deleteOnExit(); + File pigUtilFile = new File(controllerFile.getParent() + "/pig_util.py"); + pigUtilFile.deleteOnExit(); + InputStream pythonUtilStream = this.getClass().getResourceAsStream(PYTHON_PIG_UTIL_PATH); + try { + FileUtils.copyInputStreamToFile(pythonUtilStream, pigUtilFile); + } finally { + pythonUtilStream.close(); + } + controllerPath = controllerFile.getAbsolutePath(); + } + return controllerPath; } @Override public Datum eval(Tuple tuple) { - return null; + return getOutput(tuple); + } + + private Datum getOutput(Tuple input) { + if (outputQueue == null) { + throw new RuntimeException("Process has already been shut down. No way to retrieve output for input: " + input); + } + +// if (ScriptingOutputCapturer.isClassCapturingOutput() && +// !soc.isInstanceCapturingOutput()) { +// Tuple t = TupleFactory.getInstance().newTuple(TURN_ON_OUTPUT_CAPTURING); +// try { +// inputQueue.put(t); +// } catch (InterruptedException e) { +// throw new RuntimeException("Failed adding capture input flag to inputQueue"); +// } +// soc.setInstanceCapturingOutput(true); +// } + + try { + if (this.inSchema == null || this.inSchema.size() == 0) { + //When nothing is passed into the UDF the tuple + //being sent is the full tuple for the relation. + //We want it to be nothing (since that's what the user wrote). +// input = TupleFactory.getInstance().newTuple(0); + input = new VTuple(0); + } + inputQueue.put(input); + } catch (Exception e) { + throw new RuntimeException("Failed adding input to inputQueue", e); + } + Object o = null; + try { + if (outputQueue != null) { + o = outputQueue.take(); + if (o == NULL_OBJECT) { + o = null; + } + } + } catch (Exception e) { + throw new RuntimeException("Problem getting output", e); + } + + if (o == ERROR_OUTPUT) { + outputQueue = null; + if (outerrThreadsError == null) { + outerrThreadsError = new StreamingUDFException("python", "Problem with streaming udf. Can't recreate exception."); + } + throw new RuntimeException(outerrThreadsError); + } + + return JythonUtils.objectToDatum(outSchema.getColumn(0).getDataType(), o); } /** @@ -139,8 +418,8 @@ public void run() { //an error from the error thread. if (outerrThreadsError == null) { outerrThreadsError = new StreamingUDFException( - language, "Error deserializing output. Please check that the declared outputSchema for function " + - funcName + " matches the data type being returned.", e); + "python", "Error deserializing output. Please check that the declared outputSchema for function " + + invocationDesc.getName() + " matches the data type being returned.", e); } outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. } catch(InterruptedException ie) { @@ -177,7 +456,7 @@ public void run() { error.append(errInput + "\n"); } } - outerrThreadsError = new StreamingUDFException(language, error.toString(), lineNumber); + outerrThreadsError = new StreamingUDFException("python", error.toString(), lineNumber); if (outputQueue != null) { outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java new file mode 100644 index 0000000000..7f5002f223 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java @@ -0,0 +1,125 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.expr; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.plan.function.stream.StreamingCommand; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class StreamingUtil { + + private static Log LOG = LogFactory.getLog(StreamingUtil.class); + + private static final String BASH = "bash"; + private static final String PATH = "PATH"; + + /** + * Create an external process for StreamingCommand command. + * + * @param command + * @return + */ + public static ProcessBuilder createProcess(OverridableConf queryContext, StreamingCommand command) { + // Set the actual command to run with 'bash -c exec ...' + List cmdArgs = new ArrayList(); + String[] argv = command.getCommandArgs(); + + StringBuffer argBuffer = new StringBuffer(); + for (String arg : argv) { + argBuffer.append(arg); + argBuffer.append(" "); + } + String argvAsString = argBuffer.toString(); + + if (System.getProperty("os.name").toUpperCase().startsWith("WINDOWS")) { + cmdArgs.add("cmd"); + cmdArgs.add("/c"); + cmdArgs.add(argvAsString); + } else { + cmdArgs.add(BASH); + cmdArgs.add("-c"); + StringBuffer sb = new StringBuffer(); + sb.append("exec "); + sb.append(argvAsString); + cmdArgs.add(sb.toString()); + } + + // Start the external process + ProcessBuilder processBuilder = new ProcessBuilder(cmdArgs + .toArray(new String[cmdArgs.size()])); + setupEnvironment(queryContext, processBuilder); + return processBuilder; + } + + /** + * Set up the run-time environment of the managed process. + * + * @param pb + * {@link ProcessBuilder} used to exec the process + */ + private static void setupEnvironment(OverridableConf queryContext, ProcessBuilder pb) { + String separator = ":"; + Map env = pb.environment(); +// addJobConfToEnvironment(queryContext, env); + + // Add the current-working-directory to the $PATH + File dir = pb.directory(); + String cwd = (dir != null) ? dir.getAbsolutePath() : System + .getProperty("user.dir"); + + String envPath = env.get(PATH); + if (envPath == null) { + envPath = cwd; + } else { + envPath = envPath + separator + cwd; + } + env.put(PATH, envPath); + } + +// protected static void addJobConfToEnvironment(OverridableConf queryContext, Map env) { +// String propsToSend = queryContext.get(PIG_STREAMING_ENVIRONMENT); +// LOG.debug("Properties to ship to streaming environment set in "+PIG_STREAMING_ENVIRONMENT+": " + propsToSend); +// if (propsToSend == null) { +// return; +// } +// +// for (String prop : propsToSend.split(",")) { +// String value = conf.get(prop); +// if (value == null) { +// LOG.warn("Property set in "+PIG_STREAMING_ENVIRONMENT+" not found in Configuration: " + prop); +// continue; +// } +// LOG.debug("Setting property in streaming environment: " + prop); +// envPut(env, prop, value); +// } +// } + + private static void envPut(Map env, String name, String value) { + if (LOG.isDebugEnabled()) { + LOG.debug("Add env entry:" + name + "=" + value); + } + env.put(name, value); + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java index 6940b963d3..0888cf61f6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java @@ -1,315 +1,315 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.python; - -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.exception.UnsupportedException; -import org.apache.tajo.function.FunctionInvocation; -import org.apache.tajo.function.FunctionSignature; -import org.apache.tajo.function.FunctionSupplement; -import org.apache.tajo.function.PythonInvocationDesc; -import org.apache.tajo.util.TUtil; -import org.python.core.*; -import org.python.util.PythonInterpreter; - -/** - * Implementation of the script engine for Jython - */ -public class JythonScriptEngine extends TajoScriptEngine { - private static final Log LOG = LogFactory.getLog(JythonScriptEngine.class); - public static final String NAMESPACE_SEPARATOR = "."; - - /** - * Language Interpreter Uses static holder pattern - */ - private static class Interpreter { - static final PythonInterpreter interpreter; - static final ArrayList filesLoaded = new ArrayList(); - static final String JVM_JAR; - - static { - // should look like: file:JVM_JAR!/java/lang/Object.class - String rpath = Object.class.getResource("Object.class").getPath(); - JVM_JAR = rpath.replaceAll("^file:(.*)!/java/lang/Object.class$", "$1"); - - // Determine if a usable python.cachedir has been provided - // if not, certain uses of jython's import will not work e.g., so create a tmp dir - // - from some.package import * - // - import non.jvm.package - try { - String skip = System.getProperty(PySystemState.PYTHON_CACHEDIR_SKIP, "false"); - if (skip.equalsIgnoreCase("true")) { - LOG.warn("jython cachedir skipped, jython may not work"); - } else { - File tmp = null; - String cdir = System.getProperty(PySystemState.PYTHON_CACHEDIR); - if (cdir != null) { - tmp = new File(cdir); - if (!tmp.canWrite()) { - LOG.error("CACHEDIR: not writable"); - throw new RuntimeException("python.cachedir not writable: " + cdir); - } - } - if (tmp == null) { - tmp = File.createTempFile("tajo_jython_", ""); - tmp.delete(); - if (!tmp.mkdirs()) { - LOG.warn("unable to create a tmp dir for the cache, jython may not work"); - } else { - LOG.info("created tmp python.cachedir=" + tmp); - System.setProperty(PySystemState.PYTHON_CACHEDIR, tmp.getAbsolutePath()); - } - Runtime.getRuntime().addShutdownHook(new DirDeleter(tmp)); - } - } - // local file system import path elements: current dir, JYTHON_HOME/Lib - Py.getSystemState().path.append(new PyString(System.getProperty("user.dir"))); - String jyhome = System.getenv("JYTHON_HOME"); - if (jyhome != null) { - Py.getSystemState().path.append(new PyString(jyhome + File.separator + "Lib")); - } - } catch (Exception e) { - LOG.warn("issue with jython cache dir", e); - } - - // cacdedir now configured, allocate the python interpreter - interpreter = new PythonInterpreter(); - } - - /** - * Ensure the decorator functions are defined in the interpreter, and - * manage the module import dependencies. - * @param initPhase True if the script is not registered. Otherwise false. - * @param path location of a script file to exec in the interpreter - * @throws IOException - */ - static synchronized void init(boolean initPhase, String path) throws IOException { - // Decorators - - // "outputType" - - if (!filesLoaded.contains(path)) { - // attempt addition of type decorator handler, fail silently - interpreter.exec("def outputType(type_def):\n" - + " def decorator(func):\n" - + " func.outputType = type_def\n" - + " return func\n" - + " return decorator\n\n"); - - // TODO: Currently, we don't support the customized output type feature. -// interpreter.exec("def outputSchemaFunction(schema_def):\n" +///* +// * Licensed to the Apache Software Foundation (ASF) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. The ASF licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// +//package org.apache.tajo.plan.function.python; +// +//import java.io.File; +//import java.io.IOException; +//import java.io.InputStream; +//import java.util.ArrayList; +//import java.util.HashMap; +//import java.util.List; +//import java.util.Map; +//import java.util.Set; +// +//import org.apache.commons.logging.Log; +//import org.apache.commons.logging.LogFactory; +//import org.apache.tajo.catalog.FunctionDesc; +//import org.apache.tajo.catalog.proto.CatalogProtos; +//import org.apache.tajo.common.TajoDataTypes; +//import org.apache.tajo.exception.UnsupportedException; +//import org.apache.tajo.function.FunctionInvocation; +//import org.apache.tajo.function.FunctionSignature; +//import org.apache.tajo.function.FunctionSupplement; +//import org.apache.tajo.function.PythonInvocationDesc; +//import org.apache.tajo.util.TUtil; +//import org.python.core.*; +//import org.python.util.PythonInterpreter; +// +///** +// * Implementation of the script engine for Jython +// */ +//public class JythonScriptEngine extends TajoScriptEngine { +// private static final Log LOG = LogFactory.getLog(JythonScriptEngine.class); +// public static final String NAMESPACE_SEPARATOR = "."; +// +// /** +// * Language Interpreter Uses static holder pattern +// */ +// private static class Interpreter { +// static final PythonInterpreter interpreter; +// static final ArrayList filesLoaded = new ArrayList(); +// static final String JVM_JAR; +// +// static { +// // should look like: file:JVM_JAR!/java/lang/Object.class +// String rpath = Object.class.getResource("Object.class").getPath(); +// JVM_JAR = rpath.replaceAll("^file:(.*)!/java/lang/Object.class$", "$1"); +// +// // Determine if a usable python.cachedir has been provided +// // if not, certain uses of jython's import will not work e.g., so create a tmp dir +// // - from some.package import * +// // - import non.jvm.package +// try { +// String skip = System.getProperty(PySystemState.PYTHON_CACHEDIR_SKIP, "false"); +// if (skip.equalsIgnoreCase("true")) { +// LOG.warn("jython cachedir skipped, jython may not work"); +// } else { +// File tmp = null; +// String cdir = System.getProperty(PySystemState.PYTHON_CACHEDIR); +// if (cdir != null) { +// tmp = new File(cdir); +// if (!tmp.canWrite()) { +// LOG.error("CACHEDIR: not writable"); +// throw new RuntimeException("python.cachedir not writable: " + cdir); +// } +// } +// if (tmp == null) { +// tmp = File.createTempFile("tajo_jython_", ""); +// tmp.delete(); +// if (!tmp.mkdirs()) { +// LOG.warn("unable to create a tmp dir for the cache, jython may not work"); +// } else { +// LOG.info("created tmp python.cachedir=" + tmp); +// System.setProperty(PySystemState.PYTHON_CACHEDIR, tmp.getAbsolutePath()); +// } +// Runtime.getRuntime().addShutdownHook(new DirDeleter(tmp)); +// } +// } +// // local file system import path elements: current dir, JYTHON_HOME/Lib +// Py.getSystemState().path.append(new PyString(System.getProperty("user.dir"))); +// String jyhome = System.getenv("JYTHON_HOME"); +// if (jyhome != null) { +// Py.getSystemState().path.append(new PyString(jyhome + File.separator + "Lib")); +// } +// } catch (Exception e) { +// LOG.warn("issue with jython cache dir", e); +// } +// +// // cacdedir now configured, allocate the python interpreter +// interpreter = new PythonInterpreter(); +// } +// +// /** +// * Ensure the decorator functions are defined in the interpreter, and +// * manage the module import dependencies. +// * @param initPhase True if the script is not registered. Otherwise false. +// * @param path location of a script file to exec in the interpreter +// * @throws IOException +// */ +// static synchronized void init(boolean initPhase, String path) throws IOException { +// // Decorators - +// // "outputType" +// +// if (!filesLoaded.contains(path)) { +// // attempt addition of type decorator handler, fail silently +// interpreter.exec("def outputType(type_def):\n" // + " def decorator(func):\n" -// + " func.outputSchemaFunction = schema_def\n" +// + " func.outputType = type_def\n" // + " return func\n" -// + " return decorator\n"); +// + " return decorator\n\n"); +// +// // TODO: Currently, we don't support the customized output type feature. +//// interpreter.exec("def outputSchemaFunction(schema_def):\n" +//// + " def decorator(func):\n" +//// + " func.outputSchemaFunction = schema_def\n" +//// + " return func\n" +//// + " return decorator\n"); +//// +//// interpreter.exec("def schemaFunction(schema_def):\n" +//// + " def decorator(func):\n" +//// + " func.schemaFunction = schema_def\n" +//// + " return func\n" +//// + " return decorator\n\n"); +// +// InputStream is = getScriptAsStream(path); +// if (is == null) { +// throw new IllegalStateException("unable to create a stream for path: " + path); +// } +// try { +// execfile(initPhase, is, path); +// } finally { +// is.close(); +// } +// } +// } +// +// /** +// * does not call script.close() +// * @param initPhase True if the script is not registered. Otherwise false. +// * @param script Input stream to the script file +// * @param path Path to the script file +// * @throws Exception +// */ +// static void execfile(boolean initPhase, InputStream script, String path) throws RuntimeException { +// try { +// // exec the code, arbitrary imports are processed +// interpreter.execfile(script, path); +// } catch (PyException e) { +// if (e.match(Py.SystemExit)) { +// PyObject value = e.value; +// if (PyException.isExceptionInstance(e.value)) { +// value = value.__findattr__("code"); +// } +// if (new PyInteger(0).equals(value)) { +// LOG.info("Script invoked sys.exit(0)"); +// return; +// } +// } +// String message = "Python Error. " + e; +// throw new RuntimeException(message, e); +// } +// } +// +// static void setMain(boolean isMain) { +// if (isMain) { +// interpreter.set("__name__", "__main__"); +// } else { +// interpreter.set("__name__", "__lib__"); +// } +// } +// } +// +// /** +// * Gets the Python function object. +// * @param path Path of the jython script file containing the function. +// * @param functionName Name of the function +// * @return a function object +// * @throws IOException +// */ +// public static PyFunction getFunction(String path, String functionName) throws IOException { +// Interpreter.setMain(false); +// Interpreter.init(false, path); +// return (PyFunction) Interpreter.interpreter.get(functionName); +// } +// +// @Override +// protected String getScriptingLang() { +// return "jython"; +// } +// +// @Override +// protected Map getParamsFromVariables() throws IOException { +// PyFrame frame = Py.getFrame(); +// @SuppressWarnings("unchecked") +// List locals = ((PyStringMap) frame.getLocals()).items(); +// Map vars = new HashMap(); +// for (PyTuple item : locals) { +// String key = (String) item.get(0); +// Object obj = item.get(1); +// if (obj != null) { +// String value = item.get(1).toString(); +// vars.put(key, value); +// } +// } +// return vars; +// } +// +// /** +// * File.deleteOnExit(File) does not work for a non-empty directory. This +// * Thread is used to clean up the python.cachedir (if it was a tmp dir +// * created by the Engine) +// */ +// private static class DirDeleter extends Thread { +// private final File dir; +// public DirDeleter(final File file) { +// dir = file; +// } +// @Override +// public void run() { +// try { +// delete(dir); +// } catch (Exception e) { +// LOG.warn("on cleanup", e); +// } +// } +// private static boolean delete(final File file) { +// if (file.isDirectory()) { +// for (File f : file.listFiles()) { +// delete(f); +// } +// } +// return file.delete(); +// } +// } +// +// public static Set registerFunctions(String path, String namespace) +// throws IOException { +// Interpreter.setMain(false); +// Interpreter.init(true, path); +// PythonInterpreter pi = Interpreter.interpreter; +// @SuppressWarnings("unchecked") +// List locals = ((PyStringMap) pi.getLocals()).items(); +// namespace = (namespace == null) ? "" : namespace + NAMESPACE_SEPARATOR; +// Set functionDescs = TUtil.newHashSet(); +// +// for (PyTuple item : locals) { +// String key = (String) item.get(0); +// Object value = item.get(1); +// if (!key.startsWith(JythonConstants.SKIP_TOKEN) && !key.equals(JythonConstants.SCHEMA_FUNCTION) +// && !key.equals(JythonConstants.OUTPUT_TYPE) +// && !key.equals(JythonConstants.OUTPUT_SCHEMA_FUNCTION) +// && (value instanceof PyFunction) +// && (((PyFunction)value).__findattr__(JythonConstants.SCHEMA_FUNCTION)== null)) { +// PyFunction pyFunction = (PyFunction) value; +// +// // Find the pre-defined output schema +// TajoDataTypes.Type returnType; +// PyObject obj = pyFunction.__findattr__(JythonConstants.OUTPUT_TYPE); +// if (obj != null) { +// returnType = pyObjectToType(obj); +// } else { +// // the default return type is the byte array +// returnType = TajoDataTypes.Type.BLOB; +// } +// +// // Parameters have the ANY type. +// int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; +// TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; +// for (int i = 0; i < paramNum; i++) { +// paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); +// } +// +// FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, key, +// TajoDataTypes.DataType.newBuilder().setType(returnType).build(), paramTypes); +// FunctionInvocation invocation = new FunctionInvocation(); +// PythonInvocationDesc invocationDesc = new PythonInvocationDesc(key, path); +// invocation.setPython(invocationDesc); +// FunctionSupplement supplement = new FunctionSupplement(); +// functionDescs.add(new FunctionDesc(signature, invocation, supplement)); +// LOG.info("Register scripting UDF: " + namespace + key); +// } +// } +// +// Interpreter.setMain(true); +// return functionDescs; +// } +// +// private static TajoDataTypes.Type pyObjectToType(PyObject obj) { +// return TajoDataTypes.Type.valueOf(pyObjectToTypeStringCand(obj).toUpperCase()); +// } +// +// private static String pyObjectToTypeStringCand(PyObject obj) { +// String[] types = obj.toString().split(","); +// if (types.length > 1) { +// throw new UnsupportedException("Multiple return type is not supported"); +// } +// return types[0].trim(); +// } +//} // -// interpreter.exec("def schemaFunction(schema_def):\n" -// + " def decorator(func):\n" -// + " func.schemaFunction = schema_def\n" -// + " return func\n" -// + " return decorator\n\n"); - - InputStream is = getScriptAsStream(path); - if (is == null) { - throw new IllegalStateException("unable to create a stream for path: " + path); - } - try { - execfile(initPhase, is, path); - } finally { - is.close(); - } - } - } - - /** - * does not call script.close() - * @param initPhase True if the script is not registered. Otherwise false. - * @param script Input stream to the script file - * @param path Path to the script file - * @throws Exception - */ - static void execfile(boolean initPhase, InputStream script, String path) throws RuntimeException { - try { - // exec the code, arbitrary imports are processed - interpreter.execfile(script, path); - } catch (PyException e) { - if (e.match(Py.SystemExit)) { - PyObject value = e.value; - if (PyException.isExceptionInstance(e.value)) { - value = value.__findattr__("code"); - } - if (new PyInteger(0).equals(value)) { - LOG.info("Script invoked sys.exit(0)"); - return; - } - } - String message = "Python Error. " + e; - throw new RuntimeException(message, e); - } - } - - static void setMain(boolean isMain) { - if (isMain) { - interpreter.set("__name__", "__main__"); - } else { - interpreter.set("__name__", "__lib__"); - } - } - } - - /** - * Gets the Python function object. - * @param path Path of the jython script file containing the function. - * @param functionName Name of the function - * @return a function object - * @throws IOException - */ - public static PyFunction getFunction(String path, String functionName) throws IOException { - Interpreter.setMain(false); - Interpreter.init(false, path); - return (PyFunction) Interpreter.interpreter.get(functionName); - } - - @Override - protected String getScriptingLang() { - return "jython"; - } - - @Override - protected Map getParamsFromVariables() throws IOException { - PyFrame frame = Py.getFrame(); - @SuppressWarnings("unchecked") - List locals = ((PyStringMap) frame.getLocals()).items(); - Map vars = new HashMap(); - for (PyTuple item : locals) { - String key = (String) item.get(0); - Object obj = item.get(1); - if (obj != null) { - String value = item.get(1).toString(); - vars.put(key, value); - } - } - return vars; - } - - /** - * File.deleteOnExit(File) does not work for a non-empty directory. This - * Thread is used to clean up the python.cachedir (if it was a tmp dir - * created by the Engine) - */ - private static class DirDeleter extends Thread { - private final File dir; - public DirDeleter(final File file) { - dir = file; - } - @Override - public void run() { - try { - delete(dir); - } catch (Exception e) { - LOG.warn("on cleanup", e); - } - } - private static boolean delete(final File file) { - if (file.isDirectory()) { - for (File f : file.listFiles()) { - delete(f); - } - } - return file.delete(); - } - } - - public static Set registerFunctions(String path, String namespace) - throws IOException { - Interpreter.setMain(false); - Interpreter.init(true, path); - PythonInterpreter pi = Interpreter.interpreter; - @SuppressWarnings("unchecked") - List locals = ((PyStringMap) pi.getLocals()).items(); - namespace = (namespace == null) ? "" : namespace + NAMESPACE_SEPARATOR; - Set functionDescs = TUtil.newHashSet(); - - for (PyTuple item : locals) { - String key = (String) item.get(0); - Object value = item.get(1); - if (!key.startsWith(JythonConstants.SKIP_TOKEN) && !key.equals(JythonConstants.SCHEMA_FUNCTION) - && !key.equals(JythonConstants.OUTPUT_TYPE) - && !key.equals(JythonConstants.OUTPUT_SCHEMA_FUNCTION) - && (value instanceof PyFunction) - && (((PyFunction)value).__findattr__(JythonConstants.SCHEMA_FUNCTION)== null)) { - PyFunction pyFunction = (PyFunction) value; - - // Find the pre-defined output schema - TajoDataTypes.Type returnType; - PyObject obj = pyFunction.__findattr__(JythonConstants.OUTPUT_TYPE); - if (obj != null) { - returnType = pyObjectToType(obj); - } else { - // the default return type is the byte array - returnType = TajoDataTypes.Type.BLOB; - } - - // Parameters have the ANY type. - int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; - TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; - for (int i = 0; i < paramNum; i++) { - paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); - } - - FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, key, - TajoDataTypes.DataType.newBuilder().setType(returnType).build(), paramTypes); - FunctionInvocation invocation = new FunctionInvocation(); - PythonInvocationDesc invocationDesc = new PythonInvocationDesc(key, path); - invocation.setPython(invocationDesc); - FunctionSupplement supplement = new FunctionSupplement(); - functionDescs.add(new FunctionDesc(signature, invocation, supplement)); - LOG.info("Register scripting UDF: " + namespace + key); - } - } - - Interpreter.setMain(true); - return functionDescs; - } - - private static TajoDataTypes.Type pyObjectToType(PyObject obj) { - return TajoDataTypes.Type.valueOf(pyObjectToTypeStringCand(obj).toUpperCase()); - } - - private static String pyObjectToTypeStringCand(PyObject obj) { - String[] types = obj.toString().split(","); - if (types.length > 1) { - throw new UnsupportedException("Multiple return type is not supported"); - } - return types[0].trim(); - } -} - diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java index 130c08c674..496a3cb1f3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java @@ -90,6 +90,49 @@ public static PyTuple tupleToPyTuple(Tuple tuple) { return new PyTuple(pyTuple); } + public static Datum objectToDatum(TajoDataTypes.DataType type, Object o) { + switch (type.getType()) { + case BOOLEAN: + return DatumFactory.createBool((Boolean) o); + case INT1: + case INT2: + return DatumFactory.createInt2((Short) o); + case INT4: + return DatumFactory.createInt4((Integer) o); + case INT8: + return DatumFactory.createInt8((Long) o); + case UINT1: + case UINT2: + return DatumFactory.createInt2((Short) o); + case UINT4: + return DatumFactory.createInt4((Integer) o); + case UINT8: + return DatumFactory.createInt8((Long) o); + case FLOAT4: + return DatumFactory.createFloat4((Float) o); + case FLOAT8: + return DatumFactory.createFloat8((Double) o); + case CHAR: + return DatumFactory.createChar((Character) o); + case TEXT: + return DatumFactory.createText((String) o); + case DATE: + return DatumFactory.createDate((Integer) o); + case TIME: + return DatumFactory.createTime((Long) o); + case TIMESTAMP: + return DatumFactory.createTimestamp((Long) o); + case INTERVAL: + return DatumFactory.createInterval((Long) o); + case BLOB: + return DatumFactory.createBlob((byte[]) o); + case INET4: + return DatumFactory.createInet4((Integer) o); + default: + throw new UnsupportedException(type.toString()); + } + } + /** * Convert a PyObject to a datum. * @param object diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java index bf62fd0275..50664b4eee 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java @@ -1,3 +1,21 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.tajo.plan.function.python; import com.google.common.base.Charsets; @@ -31,9 +49,8 @@ public ScriptingOutputCapturer(OverridableConf queryContext, FunctionDesc functi this.functionDesc = functionDesc; } - public String getStandardOutputRootWriteLocation() throws IOException { -// Configuration conf = UDFContext.getUDFContext().getJobConf(); -// +// public String getStandardOutputRootWriteLocation() throws IOException { +// System.getProperty() // String jobId = conf.get(MRConfiguration.JOB_ID); // String taskId = conf.get(MRConfiguration.TASK_ID); // String hadoopLogDir = System.getProperty("yarn.app.container.log.dir"); @@ -79,11 +96,11 @@ public String getStandardOutputRootWriteLocation() throws IOException { // String taskLogDir = getTaskLogDir(jobId, taskId, hadoopLogDir); // return taskLogDir + "/"; // } - return null; - } +// return null; +// } - public String getTaskLogDir(String jobId, String taskId, String hadoopLogDir) throws IOException { - String taskLogDir = null; +// public String getTaskLogDir(String jobId, String taskId, String hadoopLogDir) throws IOException { +// String taskLogDir = null; // String defaultUserLogDir = hadoopLogDir + File.separator + "userlogs"; // // if ( new File(defaultUserLogDir + File.separator + jobId).exists() ) { @@ -100,8 +117,8 @@ public String getTaskLogDir(String jobId, String taskId, String hadoopLogDir) th // throw new IOException("Could not create directory: " + taskLogDir); // } // } - return taskLogDir; - } +// return taskLogDir; +// } public static void startCapturingOutput() { ScriptingOutputCapturer.captureOutput = true; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 626d73b255..3106a822bb 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -1,96 +1,96 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.python; - -import org.apache.hadoop.util.Shell; - -import javax.script.ScriptEngine; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.InputStream; -import java.io.IOException; -import java.util.Map; - -public abstract class TajoScriptEngine { - - /** - * Open a stream load a script locally or in the classpath - * @param scriptPath the path of the script - * @return a stream (it is the responsibility of the caller to close it) - * @throws IllegalStateException if we could not open a stream - */ - protected static InputStream getScriptAsStream(String scriptPath) { - InputStream is; - File file = new File(scriptPath); - if (file.exists()) { - try { - is = new FileInputStream(file); - } catch (FileNotFoundException e) { - throw new IllegalStateException("could not find existing file "+scriptPath, e); - } - } else { - if (Shell.WINDOWS && scriptPath.charAt(1)==':') { - scriptPath = scriptPath.charAt(0) + scriptPath.substring(2); - } - // Try system, current and context classloader. - is = ScriptEngine.class.getResourceAsStream(scriptPath); - if (is == null) { - is = getResourceUsingClassLoader(scriptPath, ScriptEngine.class.getClassLoader()); - } - if (is == null) { - is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); - } - if (is == null && !file.isAbsolute()) { - String path = "/" + scriptPath; - is = ScriptEngine.class.getResourceAsStream(path); - if (is == null) { - is = getResourceUsingClassLoader(path, ScriptEngine.class.getClassLoader()); - } - if (is == null) { - is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); - } - } - } - - if (is == null) { - throw new IllegalStateException( - "Could not initialize interpreter (from file system or classpath) with " + scriptPath); - } - return is; - } - - private static InputStream getResourceUsingClassLoader(String fullFilename, ClassLoader loader) { - if (loader != null) { - return loader.getResourceAsStream(fullFilename); - } - return null; - } - - /** - * Gets ScriptEngine classname or keyword for the scripting language - */ - protected abstract String getScriptingLang(); - - /** - * Returns a map from local variable names to their values - * @throws java.io.IOException - */ - protected abstract Map getParamsFromVariables() throws IOException; -} +///* +// * Licensed to the Apache Software Foundation (ASF) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. The ASF licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// +//package org.apache.tajo.plan.function.python; +// +//import org.apache.hadoop.util.Shell; +// +//import javax.script.ScriptEngine; +//import java.io.File; +//import java.io.FileInputStream; +//import java.io.FileNotFoundException; +//import java.io.InputStream; +//import java.io.IOException; +//import java.util.Map; +// +//public abstract class TajoScriptEngine { +// +// /** +// * Open a stream load a script locally or in the classpath +// * @param scriptPath the path of the script +// * @return a stream (it is the responsibility of the caller to close it) +// * @throws IllegalStateException if we could not open a stream +// */ +// protected static InputStream getScriptAsStream(String scriptPath) { +// InputStream is; +// File file = new File(scriptPath); +// if (file.exists()) { +// try { +// is = new FileInputStream(file); +// } catch (FileNotFoundException e) { +// throw new IllegalStateException("could not find existing file "+scriptPath, e); +// } +// } else { +// if (Shell.WINDOWS && scriptPath.charAt(1)==':') { +// scriptPath = scriptPath.charAt(0) + scriptPath.substring(2); +// } +// // Try system, current and context classloader. +// is = ScriptEngine.class.getResourceAsStream(scriptPath); +// if (is == null) { +// is = getResourceUsingClassLoader(scriptPath, ScriptEngine.class.getClassLoader()); +// } +// if (is == null) { +// is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); +// } +// if (is == null && !file.isAbsolute()) { +// String path = "/" + scriptPath; +// is = ScriptEngine.class.getResourceAsStream(path); +// if (is == null) { +// is = getResourceUsingClassLoader(path, ScriptEngine.class.getClassLoader()); +// } +// if (is == null) { +// is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); +// } +// } +// } +// +// if (is == null) { +// throw new IllegalStateException( +// "Could not initialize interpreter (from file system or classpath) with " + scriptPath); +// } +// return is; +// } +// +// private static InputStream getResourceUsingClassLoader(String fullFilename, ClassLoader loader) { +// if (loader != null) { +// return loader.getResourceAsStream(fullFilename); +// } +// return null; +// } +// +// /** +// * Gets ScriptEngine classname or keyword for the scripting language +// */ +// protected abstract String getScriptingLang(); +// +// /** +// * Returns a map from local variable names to their values +// * @throws java.io.IOException +// */ +// protected abstract Map getParamsFromVariables() throws IOException; +//} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java new file mode 100644 index 0000000000..728a4218a9 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java @@ -0,0 +1,74 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.util.internal.PlatformDependent; +import org.apache.hadoop.classification.InterfaceStability; + +/* this class is PooledBuffer holder */ +public class BufferPool { + + private static final PooledByteBufAllocator allocator; + + private BufferPool() { + } + + static { + //TODO we need determine the default params + allocator = new PooledByteBufAllocator(PlatformDependent.directBufferPreferred()); + + /* if you are finding memory leak, please enable this line */ + //ResourceLeakDetector.setLevel(ResourceLeakDetector.Level.ADVANCED); + } + + public static long maxDirectMemory() { + return PlatformDependent.maxDirectMemory(); + } + + + public synchronized static ByteBuf directBuffer(int size) { + return allocator.directBuffer(size); + } + + /** + * + * @param size the initial capacity + * @param max the max capacity + * @return allocated ByteBuf from pool + */ + public static ByteBuf directBuffer(int size, int max) { + return allocator.directBuffer(size, max); + } + + @InterfaceStability.Unstable + public static void forceRelease(ByteBuf buf) { + buf.release(buf.refCnt()); + } + + /** + * the ByteBuf will increase to writable size + * @param buf + * @param minWritableBytes required minimum writable size + */ + public static void ensureWritable(ByteBuf buf, int minWritableBytes) { + buf.ensureWritable(minWritableBytes); + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java new file mode 100644 index 0000000000..869ebe9c2f --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java @@ -0,0 +1,71 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import org.apache.hadoop.fs.ByteBufferReadable; +import org.apache.hadoop.io.IOUtils; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.ScatteringByteChannel; +import java.nio.channels.spi.AbstractInterruptibleChannel; + +public class ByteBufInputChannel extends AbstractInterruptibleChannel implements ScatteringByteChannel { + + ByteBufferReadable byteBufferReadable; + ReadableByteChannel channel; + InputStream inputStream; + + public ByteBufInputChannel(InputStream inputStream) { + if (inputStream instanceof ByteBufferReadable) { + this.byteBufferReadable = (ByteBufferReadable) inputStream; + } else { + this.channel = Channels.newChannel(inputStream); + } + + this.inputStream = inputStream; + } + + @Override + public long read(ByteBuffer[] dsts, int offset, int length) { + throw new UnsupportedOperationException(); + } + + @Override + public long read(ByteBuffer[] dsts) { + return read(dsts, 0, dsts.length); + } + + @Override + public int read(ByteBuffer dst) throws IOException { + if (byteBufferReadable != null) { + return byteBufferReadable.read(dst); + } else { + return channel.read(dst); + } + } + + @Override + protected void implCloseChannel() throws IOException { + IOUtils.cleanup(null, channel, inputStream); + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java new file mode 100644 index 0000000000..f306b360fb --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java @@ -0,0 +1,180 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBuf; +import io.netty.util.CharsetUtil; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +public class ByteBufLineReader implements Closeable { + private static int DEFAULT_BUFFER = 64 * 1024; + + private int bufferSize; + private long readBytes; + private int startIndex; + private boolean eof = false; + private ByteBuf buffer; + private final ByteBufInputChannel channel; + private final AtomicInteger lineReadBytes = new AtomicInteger(); + private final LineSplitProcessor processor = new LineSplitProcessor(); + + public ByteBufLineReader(ByteBufInputChannel channel) { + this(channel, BufferPool.directBuffer(DEFAULT_BUFFER)); + } + + public ByteBufLineReader(ByteBufInputChannel channel, ByteBuf buf) { + this.readBytes = 0; + this.channel = channel; + this.buffer = buf; + this.bufferSize = buf.capacity(); + } + + public long readBytes() { + return readBytes - buffer.readableBytes(); + } + + @Override + public void close() throws IOException { + if (this.buffer.refCnt() > 0) { + this.buffer.release(); + } + this.channel.close(); + } + + public String readLine() throws IOException { + ByteBuf buf = readLineBuf(lineReadBytes); + if (buf != null) { + return buf.toString(CharsetUtil.UTF_8); + } + return null; + } + + private void fillBuffer() throws IOException { + + int tailBytes = 0; + if (this.readBytes > 0) { + //startIndex = 0, readIndex = tailBytes length, writable = (buffer capacity - tailBytes) + this.buffer.markReaderIndex(); + this.buffer.discardReadBytes(); // compact the buffer + tailBytes = this.buffer.writerIndex(); + if (!this.buffer.isWritable()) { + // a line bytes is large than the buffer + BufferPool.ensureWritable(buffer, bufferSize * 2); + this.bufferSize = buffer.capacity(); + } + this.startIndex = 0; + } + + boolean release = true; + try { + int readBytes = tailBytes; + for (; ; ) { + int localReadBytes = buffer.writeBytes(channel, this.bufferSize - readBytes); + if (localReadBytes < 0) { + if (buffer.isWritable()) { + //if read bytes is less than the buffer capacity, there is no more bytes in the channel + eof = true; + } + break; + } + readBytes += localReadBytes; + if (readBytes == bufferSize) { + break; + } + } + this.readBytes += (readBytes - tailBytes); + release = false; + + this.buffer.readerIndex(this.buffer.readerIndex() + tailBytes); //skip past buffer (tail) + } finally { + if (release) { + buffer.release(); + } + } + } + + /** + * Read a line terminated by one of CR, LF, or CRLF. + */ + public ByteBuf readLineBuf(AtomicInteger reads) throws IOException { + int readBytes = 0; // newline + text line bytes + int newlineLength = 0; //length of terminating newline + int readable; + + this.startIndex = buffer.readerIndex(); + + loop: + while (true) { + readable = buffer.readableBytes(); + if (readable <= 0) { + buffer.readerIndex(this.startIndex); + fillBuffer(); //compact and fill buffer + + //if buffer.writerIndex() is zero, there is no bytes in buffer + if (!buffer.isReadable() && buffer.writerIndex() == 0) { + reads.set(0); + return null; + } else { + //skip first newLine + if (processor.isPrevCharCR() && buffer.getByte(buffer.readerIndex()) == LineSplitProcessor.LF) { + buffer.skipBytes(1); + if(eof && !buffer.isReadable()) { + reads.set(1); + return null; + } + + newlineLength++; + readBytes++; + startIndex = buffer.readerIndex(); + } + } + readable = buffer.readableBytes(); + } + + int endIndex = buffer.forEachByte(buffer.readerIndex(), readable, processor); + if (endIndex < 0) { + //does not appeared terminating newline + buffer.readerIndex(buffer.writerIndex()); // set to end buffer + if(eof){ + readBytes += (buffer.readerIndex() - startIndex); + break loop; + } + } else { + buffer.readerIndex(endIndex + 1); + readBytes += (buffer.readerIndex() - startIndex); //past newline + text line + + //appeared terminating CRLF + if (processor.isPrevCharCR() && buffer.isReadable() + && buffer.getByte(buffer.readerIndex()) == LineSplitProcessor.LF) { + buffer.skipBytes(1); + readBytes++; + newlineLength += 2; + } else { + newlineLength += 1; + } + break loop; + } + } + reads.set(readBytes); + return buffer.slice(startIndex, readBytes - newlineLength); + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java deleted file mode 100644 index 6f3aa73459..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/DefaultInputHandler.java +++ /dev/null @@ -1,36 +0,0 @@ -package org.apache.tajo.plan.function.stream; - -import org.apache.tajo.catalog.Schema; - -import java.io.IOException; - -public class DefaultInputHandler extends InputHandler { - - public DefaultInputHandler(Schema schema) { - serializer = RowStoreUtil.createEncoder(schema); - } - - public DefaultInputHandler(HandleSpec spec) { - serializer = (PigToStream)PigContext.instantiateFuncFromSpec(spec.spec); - } - - @Override - public InputType getInputType() { - return InputType.SYNCHRONOUS; - } - - @Override - public synchronized void close(Process process) throws IOException { - try { - super.close(process); - } catch(IOException e) { - // check if we got an exception because - // the process actually completed and we were - // trying to flush and close it's stdin - if (process == null || process.exitValue() != 0) { - // the process had not terminated normally - // throw the exception we got - throw e; - } - } - } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java index c4a2b3d525..373d083293 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -1,3 +1,21 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.tajo.plan.function.stream; import org.apache.tajo.storage.Tuple; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/LineSplitProcessor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/LineSplitProcessor.java new file mode 100644 index 0000000000..041514102f --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/LineSplitProcessor.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBufProcessor; + +public class LineSplitProcessor implements ByteBufProcessor { + public static final byte CR = '\r'; + public static final byte LF = '\n'; + private boolean prevCharCR = false; //true of prev char was CR + + @Override + public boolean process(byte value) throws Exception { + switch (value) { + case LF: + return false; + case CR: + prevCharCR = true; + return false; + default: + prevCharCR = false; + return true; + } + } + + public boolean isPrevCharCR() { + return prevCharCR; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java new file mode 100644 index 0000000000..83bab23544 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java @@ -0,0 +1,161 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import com.google.common.base.Charsets; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; +import java.io.InputStream; + +/** + * {@link OutputHandler} is responsible for handling the output of the + * Pig-Streaming external command. + * + * The output of the managed executable could be fetched in a + * {@link OutputType#SYNCHRONOUS} manner via its stdout or in an + * {@link OutputType#ASYNCHRONOUS} manner via an external file to which the + * process wrote its output. + */ +public abstract class OutputHandler { + public static final Object END_OF_OUTPUT = new Object(); + private static final byte[] DEFAULT_RECORD_DELIM = new byte[] {'\n'}; + + public enum OutputType {SYNCHRONOUS, ASYNCHRONOUS} + + protected RowStoreUtil.RowStoreDecoder deserializer; + + protected ByteBufLineReader in = null; + +// private Text currValue = new Text(); + private String currValue = null; + +// private BufferedPositionedInputStream istream; + private InputStream istream; + + //Both of these ignore the trailing \n. So if the + //default delimiter is "\n" recordDelimStr is "". + private String recordDelimStr = null; + private int recordDelimLength = 0; + + /** + * Get the handled OutputType. + * @return the handled OutputType + */ + public abstract OutputType getOutputType(); + + // flag to mark if close() has already been called + protected boolean alreadyClosed = false; + + /** + * Bind the OutputHandler to the InputStream + * from which to read the output data of the managed process. + * + * @param is InputStream from which to read the output data + * of the managed process + * @throws IOException + */ + public void bindTo(String fileName, InputStream is, + long offset, long end) throws IOException { + this.istream = is; + this.in = new ByteBufLineReader(new ByteBufInputChannel(istream)); + } + + /** + * Get the next output Tuple of the managed process. + * + * @return the next output Tuple of the managed process + * @throws IOException + */ + public Tuple getNext() throws IOException { + if (in == null) { + return null; + } + + currValue = null; + if (!readValue()) { + return null; + } + byte[] newBytes = new byte[currValue.length()]; + System.arraycopy(currValue.getBytes(), 0, newBytes, 0, currValue.length()); + return deserializer.toTuple(newBytes); + } + + private boolean readValue() throws IOException { + currValue = in.readLine(); + if (currValue == null) { + return false; + } + + while(!isEndOfRow()) { + //Need to add back the newline character we ate. +// currValue.append(new byte[] {'\n'}, 0, 1); + currValue += '\n'; + + byte[] lineBytes = readNextLine(); + if (lineBytes == null) { + //We have no more input, so just break; + break; + } +// currValue.append(lineBytes, 0, lineBytes.length); + currValue += new String(lineBytes); + } + + return true; + } + + private byte[] readNextLine() throws IOException { +// Text line = new Text(); + String line = in.readLine(); + if (line == null) { + return null; + } + + return line.getBytes(); + } + + private boolean isEndOfRow() { + if (recordDelimStr == null) { + byte[] recordDelimBa = getRecordDelimiter(); + recordDelimLength = recordDelimBa.length - 1; //Ignore trailing \n + recordDelimStr = new String(recordDelimBa, 0, recordDelimLength, Charsets.UTF_8); + } + if (recordDelimLength == 0 || currValue.length() < recordDelimLength) { + return true; + } + return currValue.contains(recordDelimStr); + } + + protected byte[] getRecordDelimiter() { + return DEFAULT_RECORD_DELIM; + } + + /** + * Close the OutputHandler. + * @throws IOException + */ + public synchronized void close() throws IOException { + if(!alreadyClosed) { + istream.close(); + istream = null; + alreadyClosed = true; + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java index 0d71a51ccc..44906cd0f6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java @@ -1,3 +1,21 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.tajo.plan.function.stream; import org.apache.tajo.catalog.Column; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java new file mode 100644 index 0000000000..1341785dc0 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java @@ -0,0 +1,477 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import org.apache.tajo.util.TUtil; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.*; + +/** + * {@link StreamingCommand} represents the specification of an external + * command to be executed in a Pig Query. + * + * StreamingCommand encapsulates all relevant details of the + * command specified by the user either directly via the STREAM + * operator or indirectly via a DEFINE operator. It includes + * details such as input/output/error specifications and also files to be + * shipped to the cluster and files to be cached. + */ +public class StreamingCommand implements Serializable, Cloneable { + private static final long serialVersionUID = 1L; + + // External command to be executed and it's parsed components + String executable; + String[] argv; + + // Files to be shipped to the cluster in-order to be executed + List shipSpec = new LinkedList(); + + // Files to be shipped to the cluster in-order to be executed + List cacheSpec = new LinkedList(); + + /** + * Handle to communicate with the external process. + */ + public enum Handle {INPUT, OUTPUT} + + /** + * Map from the the stdin/stdout/stderr handles to their specifications + */ + Map> handleSpecs = + new TreeMap>(); + + // Should the stderr of the process be persisted? + boolean persistStderr = false; + + // Directory where the process's stderr logs should be persisted. + String logDir; + + // Limit on the number of persisted log-files + int logFilesLimit = 100; + public static final int MAX_TASKS = 100; + + boolean shipFiles = true; + + /** + * Create a new StreamingCommand with the given command. + * + * @param argv parsed arguments of the command + */ + public StreamingCommand(String[] argv) { + this.argv = argv; + + // Assume that argv[0] is the executable + this.executable = this.argv[0]; + } + + /** + * Get the command to be executed. + * + * @return the command to be executed + */ + public String getExecutable() { + return executable; + } + + /** + * Set the executable for the StreamingCommand. + * + * @param executable the executable for the StreamingCommand + */ + public void setExecutable(String executable) { + this.executable = executable; + } + + /** + * Set the command line arguments for the StreamingCommand. + * + * @param argv the command line arguments for the + * StreamingCommand + */ + public void setCommandArgs(String[] argv) { + this.argv = argv; + } + + /** + * Get the parsed command arguments. + * + * @return the parsed command arguments as String[] + */ + public String[] getCommandArgs() { + return argv; + } + + /** + * Get the list of files which need to be shipped to the cluster. + * + * @return the list of files which need to be shipped to the cluster + */ + public List getShipSpecs() { + return shipSpec; + } + + /** + * Get the list of files which need to be cached on the execute nodes. + * + * @return the list of files which need to be cached on the execute nodes + */ + public List getCacheSpecs() { + return cacheSpec; + } + + /** + * Add a file to be shipped to the cluster. + * + * Users can use this to distribute executables and other necessary files + * to the clusters. + * + * @param path path of the file to be shipped to the cluster + */ + public void addPathToShip(String path) throws IOException { + // Validate + File file = new File(path); + if (!file.exists()) { + throw new IOException("Invalid ship specification: '" + path + + "' does not exist!"); + } else if (file.isDirectory()) { + throw new IOException("Invalid ship specification: '" + path + + "' is a directory and can't be shipped!"); + } + shipSpec.add(path); + } + + /** + * Attach a {@link HandleSpec} to a given {@link Handle} + * @param handle Handle to which the specification is to + * be attached. + * @param handleSpec HandleSpec for the given handle. + */ + public void addHandleSpec(Handle handle, HandleSpec handleSpec) { + List handleSpecList = handleSpecs.get(handle); + + if (handleSpecList == null) { + handleSpecList = new LinkedList(); + handleSpecs.put(handle, handleSpecList); + } + + handleSpecList.add(handleSpec); + } + + /** + * Set the input specification for the StreamingCommand. + * + * @param spec input specification + */ + public void setInputSpec(HandleSpec spec) { + List inputSpecs = getHandleSpecs(Handle.INPUT); + if (inputSpecs == null || inputSpecs.size() == 0) { + addHandleSpec(Handle.INPUT, spec); + } else { + inputSpecs.set(0, spec); + } + } + +// /** +// * Get the input specification of the StreamingCommand. +// * +// * @return input specification of the StreamingCommand +// */ +// public HandleSpec getInputSpec() { +// List inputSpecs = getHandleSpecs(Handle.INPUT); +// if (inputSpecs == null || inputSpecs.size() == 0) { +// addHandleSpec(Handle.INPUT, new HandleSpec("stdin", PigStreaming.class.getName())); +// } +// return getHandleSpecs(Handle.INPUT).get(0); +// } + + /** + * Set the specification for the primary output of the + * StreamingCommand. + * + * @param spec specification for the primary output of the + * StreamingCommand + */ + public void setOutputSpec(HandleSpec spec) { + List outputSpecs = getHandleSpecs(Handle.OUTPUT); + if (outputSpecs == null || outputSpecs.size() == 0) { + addHandleSpec(Handle.OUTPUT, spec); + } else { + outputSpecs.set(0, spec); + } + } +// +// /** +// * Get the specification of the primary output of the +// * StreamingCommand. +// * +// * @return specification of the primary output of the +// * StreamingCommand +// */ +// public HandleSpec getOutputSpec() { +// List outputSpecs = getHandleSpecs(Handle.OUTPUT); +// if (outputSpecs == null || outputSpecs.size() == 0) { +// addHandleSpec(Handle.OUTPUT, new HandleSpec("stdout", PigStreaming.class.getName())); +// } +// return getHandleSpecs(Handle.OUTPUT).get(0); +// } + + /** + * Get specifications for the given Handle. + * + * @param handle Handle of the stream + * @return specification for the given Handle + */ + public List getHandleSpecs(Handle handle) { + return handleSpecs.get(handle); + } + + /** + * Should the stderr of the managed process be persisted? + * + * @return true if the stderr of the managed process should be + * persisted, false otherwise. + */ + public boolean getPersistStderr() { + return persistStderr; + } + + /** + * Specify if the stderr of the managed process should be persisted. + * + * @param persistStderr true if the stderr of the managed + * process should be persisted, else false + */ + public void setPersistStderr(boolean persistStderr) { + this.persistStderr = persistStderr; + } + + /** + * Get the directory where the log-files of the command are persisted. + * + * @return the directory where the log-files of the command are persisted + */ + public String getLogDir() { + return logDir; + } + + /** + * Set the directory where the log-files of the command are persisted. + * + * @param logDir the directory where the log-files of the command are persisted + */ + public void setLogDir(String logDir) { + this.logDir = logDir; + if (this.logDir.startsWith("/")) { + this.logDir = this.logDir.substring(1); + } + setPersistStderr(true); + } + + /** + * Get the maximum number of tasks whose stderr logs files are persisted. + * + * @return the maximum number of tasks whose stderr logs files are persisted + */ + public int getLogFilesLimit() { + return logFilesLimit; + } + + /** + * Set the maximum number of tasks whose stderr logs files are persisted. + * @param logFilesLimit the maximum number of tasks whose stderr logs files + * are persisted + */ + public void setLogFilesLimit(int logFilesLimit) { + this.logFilesLimit = Math.min(MAX_TASKS, logFilesLimit); + } + + /** + * Set whether files should be shipped or not. + * + * @param shipFiles true if files of this command should be + * shipped, false otherwise + */ + public void setShipFiles(boolean shipFiles) { + this.shipFiles = shipFiles; + } + + /** + * Get whether files for this command should be shipped or not. + * + * @return true if files of this command should be shipped, + * false otherwise + */ + public boolean getShipFiles() { + return shipFiles; + } + +// public String toString() { +// StringBuffer sb = new StringBuffer(); +// for (String arg : getCommandArgs()) { +// sb.append(arg); +// sb.append(" "); +// } +// sb.append("(" + getInputSpec().toString() + "/"+getOutputSpec() + ")"); +// +// return sb.toString(); +// } + + public Object clone() { + try { + StreamingCommand clone = (StreamingCommand)super.clone(); + + clone.shipSpec = new ArrayList(shipSpec); + clone.cacheSpec = new ArrayList(cacheSpec); + + clone.handleSpecs = new HashMap>(); + for (Map.Entry> e : handleSpecs.entrySet()) { + List values = new ArrayList(); + for (HandleSpec spec : e.getValue()) { + values.add((HandleSpec)spec.clone()); + } + clone.handleSpecs.put(e.getKey(), values); + } + + return clone; + } catch (CloneNotSupportedException cnse) { + // Shouldn't happen since we do implement Clonable + throw new InternalError(cnse.toString()); + } + } + + + /** + * Specification about the usage of the {@link Handle} to communicate + * with the external process. + * + * It specifies the stream-handle which can be one of stdin/ + * stdout/stderr or a named file and also the + * serializer/deserializer specification to be used to read/write data + * to/from the stream. + */ + public static class HandleSpec + implements Comparable, Serializable, Cloneable { + private static final long serialVersionUID = 1L; + + String name; +// String spec; + +// /** +// * Create a new {@link HandleSpec} with a given name using the default +// * {@link PigStorage} serializer/deserializer. +// * +// * @param handleName name of the handle (one of stdin, +// * stdout or a file-path) +// */ +// public HandleSpec(String handleName) { +// this(handleName, PigStreaming.class.getName()); +// } + +// /** +// * Create a new {@link HandleSpec} with a given name using the default +// * {@link PigStorage} serializer/deserializer. +// * +// * @param handleName name of the handle (one of stdin, +// * stdout or a file-path) +// * @param spec serializer/deserializer spec +// */ +// public HandleSpec(String handleName, String spec) { +// this.name = handleName; +// this.spec = spec; +// } + + public HandleSpec(String handleName) { + this.name = handleName; + } + + public int compareTo(HandleSpec o) { + return this.name.compareTo(o.name); + } + + public String toString() { +// return name + "-" + spec; + return name; + } + + /** + * Get the name of the HandleSpec. + * + * @return the name of the HandleSpec (one of + * stdin, stdout or a file-path) + */ + public String getName() { + return name; + } + + /** + * Set the name of the HandleSpec. + * + * @param name name of the HandleSpec (one of + * stdin, stdout or a file-path) + */ + public void setName(String name) { + this.name = name; + } + +// /** +// * Get the serializer/deserializer spec of the HandleSpec. +// * +// * @return the serializer/deserializer spec of the +// * HandleSpec +// */ +// public String getSpec() { +// return spec; +// } +// +// /** +// * Set the serializer/deserializer spec of the HandleSpec. +// * +// * @param spec the serializer/deserializer spec of the +// * HandleSpec +// */ +// public void setSpec(String spec) { +// this.spec = spec; +// } + + public boolean equals(Object obj) { + if (obj instanceof HandleSpec){ + HandleSpec other = (HandleSpec)obj; +// return (other != null && name.equals(other.name) && spec.equals(other.spec)); + return TUtil.checkEquals(name, other.name); + } else + return false; + } + + public int hashCode() { + return name.hashCode(); + } + + public Object clone() { + try { + return super.clone(); + } catch (CloneNotSupportedException cnse) { + // Shouldn't happen since we do implement Clonable + throw new InternalError(cnse.toString()); + } + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java index 706e748cfd..a9c974cb5c 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java @@ -1,3 +1,21 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.tajo.plan.function.stream; public class StreamingUDFException extends Exception { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java new file mode 100644 index 0000000000..62cd168219 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java @@ -0,0 +1,49 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import java.io.IOException; + +public class StreamingUDFInputHandler extends InputHandler { + + public StreamingUDFInputHandler(RowStoreUtil.RowStoreEncoder serializer) { + this.serializer = serializer; + } + + @Override + public InputType getInputType() { + return InputType.SYNCHRONOUS; + } + + @Override + public synchronized void close(Process process) throws IOException { + try { + super.close(process); + } catch(IOException e) { + // check if we got an exception because + // the process actually completed and we were + // trying to flush and close it's stdin + if (process == null || process.exitValue() != 0) { + // the process had not terminated normally + // throw the exception we got + throw e; + } + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java new file mode 100644 index 0000000000..122b035a24 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java @@ -0,0 +1,36 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +public class StreamingUDFOutputHandler extends OutputHandler { + + public StreamingUDFOutputHandler(RowStoreUtil.RowStoreDecoder deserializer) { + this.deserializer = deserializer; + } + + @Override + protected byte[] getRecordDelimiter() { + return ",".getBytes(); + } + + @Override + public OutputType getOutputType() { + return OutputType.SYNCHRONOUS; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java deleted file mode 100644 index 12775b860d..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StringCommand.java +++ /dev/null @@ -1,4 +0,0 @@ -package org.apache.tajo.plan.function.stream; - -public class StringCommand { -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index caa454517b..355c695e5a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -39,6 +39,7 @@ import org.apache.tajo.plan.logical.WindowSpec; import org.apache.tajo.plan.serder.PlanProto.WinFunctionEvalSpec; +import java.io.IOException; import java.util.*; /** @@ -226,6 +227,8 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { throw new NoSuchFunctionException(functionName, parameterTypes); } catch (InternalException ie) { throw new NoSuchFunctionException(funcDesc.getFunctionName(), funcDesc.getParamTypes()); + } catch (IOException e) { + throw new NoSuchFunctionException(e.getMessage()); } } else { throw new RuntimeException("Unknown EvalType: " + type.name()); From 047f50eba66e719b24ad7f1210fb1379176e9501 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 5 Apr 2015 00:28:29 +0900 Subject: [PATCH 25/55] TAJO-1344 --- .../tajo/engine/function/FunctionLoader.java | 5 +- .../tajo/plan/expr/PythonFunctionInvoke.java | 196 +++++++++--------- .../function/python/PythonScriptEngine.java | 175 ++++++++++++++++ .../function/python/TajoScriptEngine.java | 194 ++++++++--------- 4 files changed, 374 insertions(+), 196 deletions(-) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index fee066f5d1..18a61da9a1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -37,7 +37,7 @@ import org.apache.tajo.engine.function.annotation.ParamOptionTypes; import org.apache.tajo.engine.function.annotation.ParamTypes; import org.apache.tajo.function.*; -import org.apache.tajo.plan.function.python.JythonScriptEngine; +import org.apache.tajo.plan.function.python.PythonScriptEngine; import org.apache.tajo.util.ClassUtil; import org.apache.tajo.util.TUtil; @@ -95,9 +95,10 @@ public static Map loadOptionalFunctions(TajoCon filePaths.add(codePath); } for (Path filePath : filePaths) { - for (FunctionDesc f : JythonScriptEngine.registerFunctions(filePath.toString(), + for (FunctionDesc f : PythonScriptEngine.registerFunctions(filePath.toString(), FunctionLoader.PYTHON_FUNCTION_NAMESPACE)) { functionMap.put(f.getSignature(), f); +// LOG.info(f); } } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java index 3145a7770a..0313874140 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java @@ -1,98 +1,98 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.expr; - -import com.google.common.base.Objects; -import com.google.gson.annotations.Expose; -import org.apache.tajo.OverridableConf; -import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.function.PythonInvocationDesc; -import org.apache.tajo.plan.function.python.JythonScriptEngine; -import org.apache.tajo.plan.function.python.JythonUtils; -import org.apache.tajo.storage.Tuple; -import org.apache.tajo.util.TUtil; -import org.python.core.PyFunction; -import org.python.core.PyObject; - -import java.io.IOException; -import java.util.Arrays; - -public class PythonFunctionInvoke extends FunctionInvoke { - @Expose private PythonInvocationDesc invokeDesc; - @Expose private TajoDataTypes.DataType[] paramTypes; - - public PythonFunctionInvoke(FunctionDesc funcDesc) { - super(funcDesc); - this.invokeDesc = funcDesc.getInvocation().getPython(); - this.paramTypes = funcDesc.getSignature().getParamTypes(); - } - - @Override - public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { - // nothing to do - } - - @Override - public Datum eval(Tuple tuple) { - try { - PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); - - PyObject result; - if (paramTypes.length == 0) { - result = function.__call__(); - } else { - // Find the actual data types from the given parameters at runtime, - // and convert them into PyObject instances. - PyObject[] pyParams = JythonUtils.tupleToPyTuple(tuple).getArray(); - result = function.__call__(pyParams); - } - - return JythonUtils.pyObjectToDatum(result); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public boolean equals(Object o) { - if (o instanceof PythonFunctionInvoke) { - PythonFunctionInvoke other = (PythonFunctionInvoke) o; - return this.invokeDesc.equals(other.invokeDesc) && - TUtil.checkEquals(this.paramTypes, other.paramTypes); - } - return false; - } - - @Override - public int hashCode() { - return Objects.hashCode(invokeDesc, Arrays.hashCode(paramTypes)); - } - - @Override - public Object clone() throws CloneNotSupportedException { - PythonFunctionInvoke clone = (PythonFunctionInvoke) super.clone(); - clone.invokeDesc = (PythonInvocationDesc) this.invokeDesc.clone(); - clone.paramTypes = new TajoDataTypes.DataType[paramTypes.length]; - paramTypes = Arrays.copyOf(paramTypes, paramTypes.length); - return clone; - } -} +///** +// * Licensed to the Apache Software Foundation (ASF) under one +// * or more contributor license agreements. See the NOTICE file +// * distributed with this work for additional information +// * regarding copyright ownership. The ASF licenses this file +// * to you under the Apache License, Version 2.0 (the +// * "License"); you may not use this file except in compliance +// * with the License. You may obtain a copy of the License at +// * +// * http://www.apache.org/licenses/LICENSE-2.0 +// * +// * Unless required by applicable law or agreed to in writing, software +// * distributed under the License is distributed on an "AS IS" BASIS, +// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// * See the License for the specific language governing permissions and +// * limitations under the License. +// */ +// +//package org.apache.tajo.plan.expr; +// +//import com.google.common.base.Objects; +//import com.google.gson.annotations.Expose; +//import org.apache.tajo.OverridableConf; +//import org.apache.tajo.catalog.FunctionDesc; +//import org.apache.tajo.catalog.Schema; +//import org.apache.tajo.common.TajoDataTypes; +//import org.apache.tajo.datum.Datum; +//import org.apache.tajo.function.PythonInvocationDesc; +//import org.apache.tajo.plan.function.python.JythonScriptEngine; +//import org.apache.tajo.plan.function.python.JythonUtils; +//import org.apache.tajo.storage.Tuple; +//import org.apache.tajo.util.TUtil; +//import org.python.core.PyFunction; +//import org.python.core.PyObject; +// +//import java.io.IOException; +//import java.util.Arrays; +// +//public class PythonFunctionInvoke extends FunctionInvoke { +// @Expose private PythonInvocationDesc invokeDesc; +// @Expose private TajoDataTypes.DataType[] paramTypes; +// +// public PythonFunctionInvoke(FunctionDesc funcDesc) { +// super(funcDesc); +// this.invokeDesc = funcDesc.getInvocation().getPython(); +// this.paramTypes = funcDesc.getSignature().getParamTypes(); +// } +// +// @Override +// public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { +// // nothing to do +// } +// +// @Override +// public Datum eval(Tuple tuple) { +// try { +// PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); +// +// PyObject result; +// if (paramTypes.length == 0) { +// result = function.__call__(); +// } else { +// // Find the actual data types from the given parameters at runtime, +// // and convert them into PyObject instances. +// PyObject[] pyParams = JythonUtils.tupleToPyTuple(tuple).getArray(); +// result = function.__call__(pyParams); +// } +// +// return JythonUtils.pyObjectToDatum(result); +// } catch (IOException e) { +// throw new RuntimeException(e); +// } +// } +// +// @Override +// public boolean equals(Object o) { +// if (o instanceof PythonFunctionInvoke) { +// PythonFunctionInvoke other = (PythonFunctionInvoke) o; +// return this.invokeDesc.equals(other.invokeDesc) && +// TUtil.checkEquals(this.paramTypes, other.paramTypes); +// } +// return false; +// } +// +// @Override +// public int hashCode() { +// return Objects.hashCode(invokeDesc, Arrays.hashCode(paramTypes)); +// } +// +// @Override +// public Object clone() throws CloneNotSupportedException { +// PythonFunctionInvoke clone = (PythonFunctionInvoke) super.clone(); +// clone.invokeDesc = (PythonInvocationDesc) this.invokeDesc.clone(); +// clone.paramTypes = new TajoDataTypes.DataType[paramTypes.length]; +// paramTypes = Arrays.copyOf(paramTypes, paramTypes.length); +// return clone; +// } +//} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java new file mode 100644 index 0000000000..179cbaf0ef --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.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.tajo.plan.function.python; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.catalog.CatalogUtil; +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.catalog.proto.CatalogProtos; +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.function.FunctionInvocation; +import org.apache.tajo.function.FunctionSignature; +import org.apache.tajo.function.FunctionSupplement; +import org.apache.tajo.function.PythonInvocationDesc; +import org.apache.tajo.util.TUtil; + +import java.io.*; +import java.nio.charset.Charset; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Pattern; + +public class PythonScriptEngine extends TajoScriptEngine { + + private static final Log log = LogFactory.getLog(PythonScriptEngine.class); + + public static Set registerFunctions(String path, String namespace) throws IOException { + +// String command = pigContext.getProperties().getProperty( +// PigConfiguration.PIG_STREAMING_UDF_PYTHON_COMMAND, "python"); + Set functionDescs = TUtil.newHashSet(); + + String command = "python"; + String fileName = path.substring(0, path.length() - ".py".length()); + log.debug("Path: " + path + " FileName: " + fileName + " Namespace: " + namespace); +// File f = new File(path); +// +// if (!f.canRead()) { +// throw new IOException("Can't read file: " + path); +// } +// +// FileInputStream fin = new FileInputStream(f); + InputStream in = getScriptAsStream(path); + List functions = null; + try { + functions = getFunctions(in); + } finally { + in.close(); + } + namespace = namespace == null ? "" : namespace + NAMESPACE_SEPARATOR; + for(FuncInfo funcInfo : functions) { + String alias = namespace + funcInfo.funcName; +// String execType = (pigContext.getExecType() == ExecType.LOCAL? "local" : "mapreduce"); +// String isIllustrate = (Boolean.valueOf(pigContext.inIllustrator)).toString(); + log.debug("Registering Function: " + alias); +// pigContext.registerFunction(alias, +// new FuncSpec("StreamingUDF", +// new String[] { +// command, +// fileName, name, +// schemaString, schemaLineNumber, +// execType, isIllustrate +// })); + TajoDataTypes.DataType returnType = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.valueOf(funcInfo.returnType)); + FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, funcInfo.funcName, + returnType, createParamTypes(funcInfo.paramNum)); + FunctionInvocation invocation = new FunctionInvocation(); + PythonInvocationDesc invocationDesc = new PythonInvocationDesc(funcInfo.funcName, path); + invocation.setPython(invocationDesc); + FunctionSupplement supplement = new FunctionSupplement(); + functionDescs.add(new FunctionDesc(signature, invocation, supplement)); + } + return functionDescs; + } + + private static TajoDataTypes.DataType[] createParamTypes(int paramNum) { + TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; + for (int i = 0; i < paramNum; i++) { + paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); + } + return paramTypes; + } + +// @Override +// protected Map> main(PigContext context, +// String scriptFile) throws IOException { +// log.warn("ScriptFile: " + scriptFile); +// registerFunctions(scriptFile, null, context); +// return getPigStatsMap(); +// } + + @Override + protected String getScriptingLang() { + return "streaming_python"; + } + + @Override + protected Map getParamsFromVariables() throws IOException { + throw new IOException("Unsupported Operation"); + } + + private static final Pattern pSchema = Pattern.compile("^\\s*\\W+outputType.*"); + private static final Pattern pDef = Pattern.compile("^\\s*def\\s+(\\w+)\\s*.+"); + + private static class FuncInfo { + String returnType; + String funcName; + int paramNum; + int schemaLineNumber; + + public FuncInfo(String returnType, String funcName, int paramNum, int schemaLineNumber) { + this.returnType = returnType.toUpperCase(); + this.funcName = funcName; + this.paramNum = paramNum; + this.schemaLineNumber = schemaLineNumber; + } + } + + // TODO: python parser must be improved. + private static List getFunctions(InputStream is) throws IOException { + List functions = TUtil.newList(); + InputStreamReader in = new InputStreamReader(is, Charset.defaultCharset()); + BufferedReader br = new BufferedReader(in); + String line = br.readLine(); + String schemaString = null; + int lineNumber = 1; + int schemaLineNumber = -1; + while (line != null) { + if (pSchema.matcher(line).matches()) { + int start = line.indexOf("(") + 2; //drop brackets/quotes + int end = line.lastIndexOf(")") - 1; + schemaString = line.substring(start,end).trim(); + schemaLineNumber = lineNumber; + } else if (pDef.matcher(line).matches()) { + int nameStart = line.indexOf("def ") + "def ".length(); + int nameEnd = line.indexOf('('); + int signatureEnd = line.indexOf(')'); + String[] params = line.substring(nameEnd+1, signatureEnd).split(","); + int paramNum; + if (params.length == 1) { + paramNum = params[0].equals("") ? 0 : 1; + } else { + paramNum = params.length; + } + + String functionName = line.substring(nameStart, nameEnd).trim(); + schemaString = schemaString == null ? "blob" : schemaString; + functions.add(new FuncInfo(schemaString, functionName, paramNum, schemaLineNumber)); + schemaString = null; + } + line = br.readLine(); + lineNumber++; + } + br.close(); + in.close(); + return functions; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 3106a822bb..55e4115500 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -1,96 +1,98 @@ -///* -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ -// -//package org.apache.tajo.plan.function.python; -// -//import org.apache.hadoop.util.Shell; -// -//import javax.script.ScriptEngine; -//import java.io.File; -//import java.io.FileInputStream; -//import java.io.FileNotFoundException; -//import java.io.InputStream; -//import java.io.IOException; -//import java.util.Map; -// -//public abstract class TajoScriptEngine { -// -// /** -// * Open a stream load a script locally or in the classpath -// * @param scriptPath the path of the script -// * @return a stream (it is the responsibility of the caller to close it) -// * @throws IllegalStateException if we could not open a stream -// */ -// protected static InputStream getScriptAsStream(String scriptPath) { -// InputStream is; -// File file = new File(scriptPath); -// if (file.exists()) { -// try { -// is = new FileInputStream(file); -// } catch (FileNotFoundException e) { -// throw new IllegalStateException("could not find existing file "+scriptPath, e); -// } -// } else { -// if (Shell.WINDOWS && scriptPath.charAt(1)==':') { -// scriptPath = scriptPath.charAt(0) + scriptPath.substring(2); -// } -// // Try system, current and context classloader. -// is = ScriptEngine.class.getResourceAsStream(scriptPath); -// if (is == null) { -// is = getResourceUsingClassLoader(scriptPath, ScriptEngine.class.getClassLoader()); -// } -// if (is == null) { -// is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); -// } -// if (is == null && !file.isAbsolute()) { -// String path = "/" + scriptPath; -// is = ScriptEngine.class.getResourceAsStream(path); -// if (is == null) { -// is = getResourceUsingClassLoader(path, ScriptEngine.class.getClassLoader()); -// } -// if (is == null) { -// is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); -// } -// } -// } -// -// if (is == null) { -// throw new IllegalStateException( -// "Could not initialize interpreter (from file system or classpath) with " + scriptPath); -// } -// return is; -// } -// -// private static InputStream getResourceUsingClassLoader(String fullFilename, ClassLoader loader) { -// if (loader != null) { -// return loader.getResourceAsStream(fullFilename); -// } -// return null; -// } -// -// /** -// * Gets ScriptEngine classname or keyword for the scripting language -// */ -// protected abstract String getScriptingLang(); -// -// /** -// * Returns a map from local variable names to their values -// * @throws java.io.IOException -// */ -// protected abstract Map getParamsFromVariables() throws IOException; -//} +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.python; + +import org.apache.hadoop.util.Shell; + +import javax.script.ScriptEngine; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.InputStream; +import java.io.IOException; +import java.util.Map; + +public abstract class TajoScriptEngine { + + public static final String NAMESPACE_SEPARATOR = "."; + + /** + * Open a stream load a script locally or in the classpath + * @param scriptPath the path of the script + * @return a stream (it is the responsibility of the caller to close it) + * @throws IllegalStateException if we could not open a stream + */ + protected static InputStream getScriptAsStream(String scriptPath) { + InputStream is; + File file = new File(scriptPath); + if (file.exists()) { + try { + is = new FileInputStream(file); + } catch (FileNotFoundException e) { + throw new IllegalStateException("could not find existing file "+scriptPath, e); + } + } else { + if (Shell.WINDOWS && scriptPath.charAt(1)==':') { + scriptPath = scriptPath.charAt(0) + scriptPath.substring(2); + } + // Try system, current and context classloader. + is = ScriptEngine.class.getResourceAsStream(scriptPath); + if (is == null) { + is = getResourceUsingClassLoader(scriptPath, ScriptEngine.class.getClassLoader()); + } + if (is == null) { + is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); + } + if (is == null && !file.isAbsolute()) { + String path = "/" + scriptPath; + is = ScriptEngine.class.getResourceAsStream(path); + if (is == null) { + is = getResourceUsingClassLoader(path, ScriptEngine.class.getClassLoader()); + } + if (is == null) { + is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); + } + } + } + + if (is == null) { + throw new IllegalStateException( + "Could not initialize interpreter (from file system or classpath) with " + scriptPath); + } + return is; + } + + private static InputStream getResourceUsingClassLoader(String fullFilename, ClassLoader loader) { + if (loader != null) { + return loader.getResourceAsStream(fullFilename); + } + return null; + } + + /** + * Gets ScriptEngine classname or keyword for the scripting language + */ + protected abstract String getScriptingLang(); + + /** + * Returns a map from local variable names to their values + * @throws java.io.IOException + */ + protected abstract Map getParamsFromVariables() throws IOException; +} From ff08099378330b594d417a45b7f113e953e5d6b8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 5 Apr 2015 23:19:46 +0900 Subject: [PATCH 26/55] TAJO-1344 --- .../java/org/apache/tajo/datum/BlobDatum.java | 6 +- .../src/main/resources/python/controller.py | 89 ++++-- .../src/main/resources/python/tajo_util.py | 2 + .../engine/function/TestPythonFunctions.java | 12 +- .../src/test/resources/python/test_funcs.py | 2 + .../src/test/resources/python/test_funcs2.py | 2 + .../tajo/plan/expr/PythonFunctionInvoke2.java | 114 +++++-- .../apache/tajo/plan/expr/StreamingUtil.java | 18 -- .../function/python/PythonScriptEngine.java | 21 +- .../python/ScriptingOutputCapturer.java | 71 ----- .../function/stream/ByteBufLineReader.java | 12 +- .../function/stream/CSVLineDeserializer.java | 99 ++++++ .../plan/function/stream/CSVLineSerDe.java | 42 +++ .../function/stream/CSVLineSerializer.java | 117 +++++++ .../stream/FieldSerializerDeserializer.java | 36 +++ .../function/stream/FieldSplitProcessor.java | 34 ++ .../plan/function/stream/InputHandler.java | 11 +- .../plan/function/stream/OutputHandler.java | 31 +- .../plan/function/stream/RowStoreUtil.java | 290 ------------------ .../stream/StreamingUDFInputHandler.java | 2 +- .../stream/StreamingUDFOutputHandler.java | 2 +- .../TextFieldSerializerDeserializer.java | 258 ++++++++++++++++ .../function/stream/TextLineDeserializer.java | 60 ++++ .../function/stream/TextLineParsingError.java | 31 ++ .../plan/function/stream/TextLineSerDe.java | 65 ++++ .../function/stream/TextLineSerializer.java | 45 +++ 26 files changed, 998 insertions(+), 474 deletions(-) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineDeserializer.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerDe.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSerializerDeserializer.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSplitProcessor.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineDeserializer.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineParsingError.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerDe.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerializer.java diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java index 4f296a1a23..2fc923fdd6 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java @@ -23,6 +23,7 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.exception.InvalidOperationException; +import org.apache.tajo.util.TUtil; import java.nio.ByteBuffer; import java.nio.charset.Charset; @@ -130,8 +131,9 @@ public boolean equals(Object obj) { BlobDatum other = (BlobDatum) obj; initFromBytes(); other.initFromBytes(); - return bb.equals(other.bb); - } +// return bb.equals(other.bb); + return Arrays.equals(bb.array(), other.bb.array()); + } return false; } diff --git a/tajo-core/src/main/resources/python/controller.py b/tajo-core/src/main/resources/python/controller.py index a37e868f4a..4b7f31b345 100644 --- a/tajo-core/src/main/resources/python/controller.py +++ b/tajo-core/src/main/resources/python/controller.py @@ -26,7 +26,7 @@ except ImportError: USE_DATEUTIL = False -from tajo_util import write_user_exception, udf_logging +from tajo_util import write_user_exception, udf_logging, outputType FIELD_DELIMITER = ',' TUPLE_START = '(' @@ -90,7 +90,7 @@ def main(self, self.stream_error = os.fdopen(sys.stderr.fileno(), 'wb', 0) self.input_stream = sys.stdin - self.output_stream = open(output_stream_path, 'a') + self.log_stream = open(output_stream_path, 'a') sys.stderr = open(error_stream_path, 'w') is_illustrate = is_illustrate_str == "true" @@ -102,9 +102,14 @@ def main(self, logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") input_str = self.get_next_input() + logging.info('main: ' + input_str) try: + # logging.info('module: ' + module_name + ' func_name: ' + func_name) + # logging.info(globals()) + # logging.info(locals()) func = __import__(module_name, globals(), locals(), [func_name], -1).__dict__[func_name] + logging.info("imported") except: #These errors should always be caused by user code. write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) @@ -114,9 +119,10 @@ def main(self, #Only log output for illustrate after we get the flag to capture output. sys.stdout = open(os.devnull, 'w') else: - sys.stdout = self.output_stream + sys.stdout = self.log_stream while input_str != END_OF_STREAM: + logging.info('while loop') should_log = False if self.input_count == self.next_input_count_to_log: should_log = True @@ -168,12 +174,15 @@ def main(self, def get_next_input(self): input_stream = self.input_stream - output_stream = self.output_stream + log_stream = self.log_stream + logging.info('test') input_str = input_stream.readline() + logging.info('input_str: ' + input_str) while input_str.endswith(END_RECORD_DELIM) == False: line = input_stream.readline() + logging.info('line: ' + line) if line == '': input_str = '' break @@ -184,7 +193,7 @@ def get_next_input(self): if input_str == TURN_ON_OUTPUT_CAPTURING: logging.debug("Turned on Output Capturing") - sys.stdout = output_stream + sys.stdout = log_stream return self.get_next_input() if input_str == END_OF_STREAM: @@ -219,10 +228,18 @@ def deserialize_input(input_str): if len(input_str) == 0: return [] - return [_deserialize_input(param, 0, len(param)-1) for param in input_str.split(WRAPPED_PARAMETER_DELIMITER)] + logging.info('deserialize_input: ' + input_str) + # [logging.info(param) for param in input_str.split(WRAPPED_PARAMETER_DELIMITER)] + # return [_deserialize_input(param, 0, len(param)) for param in input_str.split(WRAPPED_PARAMETER_DELIMITER)] + [logging.info(param) for param in input_str.split(WRAPPED_FIELD_DELIMITER)] + return [_deserialize_input(param, 0, len(param)) for param in input_str.split(WRAPPED_FIELD_DELIMITER)] def _deserialize_input(input_str, si, ei): - if ei - si < 1: + logging.info('_deserialize_input: ' + input_str) + logging.info(si) + logging.info(ei) + len = ei - si + 1 + if len < 1: #Handle all of the cases where you can have valid empty input. if ei == si: if input_str[si] == TYPE_CHARARRAY: @@ -234,35 +251,67 @@ def _deserialize_input(input_str, si, ei): else: raise Exception("Start index %d greater than end index %d.\nInput string: %s\n, Slice: %s" % (si, ei, input_str[si:ei+1])) - first = input_str[si] - schema = input_str[si+1] if first == PRE_WRAP_DELIM else first + tokens = input_str.split(WRAPPED_PARAMETER_DELIMITER) + schema = tokens[0]; + param = tokens[1]; + + # first = input_str[si] + # schema = input_str[si+1] if first == PRE_WRAP_DELIM else first + # logging.info('first: ' + first) + logging.info('schema: ' + schema) + logging.info('param: ' + param) + + # if schema == NULL_BYTE: + # return None + # elif schema == TYPE_TUPLE or schema == TYPE_MAP or schema == TYPE_BAG: + # return _deserialize_collection(input_str, schema, si+3, ei-3) + # elif schema == TYPE_CHARARRAY: + # return unicode(input_str[si+1:ei+1], 'utf-8') + # elif schema == TYPE_BYTEARRAY: + # return bytearray(input_str[si+1:ei+1]) + # elif schema == TYPE_INTEGER: + # return int(input_str[si+1:ei+1]) + # elif schema == TYPE_LONG or schema == TYPE_BIGINTEGER: + # return long(input_str[si+1:ei+1]) + # elif schema == TYPE_FLOAT or schema == TYPE_DOUBLE or schema == TYPE_BIGDECIMAL: + # return float(input_str[si+1:ei+1]) + # elif schema == TYPE_BOOLEAN: + # return input_str[si+1:ei+1] == "true" + # elif schema == TYPE_DATETIME: + # #Format is "yyyy-MM-ddTHH:mm:ss.SSS+00:00" or "2013-08-23T18:14:03.123+ZZ" + # if USE_DATEUTIL: + # return parser.parse(input_str[si+1:ei+1]) + # else: + # #Try to use datetime even though it doesn't handle time zones properly, + # #We only use the first 3 microsecond digits and drop time zone (first 23 characters) + # return datetime.strptime(input_str[si+1:si+24], "%Y-%m-%dT%H:%M:%S.%f") + # else: + # raise Exception("Can't determine type of input: %s" % input_str[si:ei+1]) if schema == NULL_BYTE: return None - elif schema == TYPE_TUPLE or schema == TYPE_MAP or schema == TYPE_BAG: - return _deserialize_collection(input_str, schema, si+3, ei-3) elif schema == TYPE_CHARARRAY: - return unicode(input_str[si+1:ei+1], 'utf-8') + return unicode(param, 'utf-8') elif schema == TYPE_BYTEARRAY: - return bytearray(input_str[si+1:ei+1]) + return bytearray(param) elif schema == TYPE_INTEGER: - return int(input_str[si+1:ei+1]) + return int(param) elif schema == TYPE_LONG or schema == TYPE_BIGINTEGER: - return long(input_str[si+1:ei+1]) + return long(param) elif schema == TYPE_FLOAT or schema == TYPE_DOUBLE or schema == TYPE_BIGDECIMAL: - return float(input_str[si+1:ei+1]) + return float(param) elif schema == TYPE_BOOLEAN: - return input_str[si+1:ei+1] == "true" + return param == "true" elif schema == TYPE_DATETIME: #Format is "yyyy-MM-ddTHH:mm:ss.SSS+00:00" or "2013-08-23T18:14:03.123+ZZ" if USE_DATEUTIL: - return parser.parse(input_str[si+1:ei+1]) + return parser.parse(param) else: #Try to use datetime even though it doesn't handle time zones properly, #We only use the first 3 microsecond digits and drop time zone (first 23 characters) - return datetime.strptime(input_str[si+1:si+24], "%Y-%m-%dT%H:%M:%S.%f") + return datetime.strptime(param, "%Y-%m-%dT%H:%M:%S.%f") else: - raise Exception("Can't determine type of input: %s" % input_str[si:ei+1]) + raise Exception("Can't determine type of input: %s" % param) def _deserialize_collection(input_str, return_type, si, ei): list_result = [] diff --git a/tajo-core/src/main/resources/python/tajo_util.py b/tajo-core/src/main/resources/python/tajo_util.py index f97e0c5bea..77b28a6980 100644 --- a/tajo-core/src/main/resources/python/tajo_util.py +++ b/tajo-core/src/main/resources/python/tajo_util.py @@ -16,6 +16,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +import logging + class udf_logging(object): udf_log_level = logging.INFO diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java index 869ffb65fa..668cfadc54 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -27,12 +27,12 @@ public class TestPythonFunctions extends ExprTestBase { @Test public void test() throws IOException { - testSimpleEval("select return_one()", new String[]{"1"}); +// testSimpleEval("select return_one()", new String[]{"1"}); // testSimpleEval("select helloworld()", new String[]{"Hello, World"}); -// testSimpleEval("select sum_py(1,2)", new String[]{"3"}); -// testSimpleEval("select concat_py('1')", new String[]{"11"}); -// testSimpleEval("select comma_format(12345)", new String[]{"12,345"}); -// testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); -// testSimpleEval("select percent(386, 1000)", new String[]{"38.6"}); + testSimpleEval("select sum_py(1,2)", new String[]{"3"}); + testSimpleEval("select concat_py('1')", new String[]{"11"}); + testSimpleEval("select comma_format(12345)", new String[]{"12,345"}); + testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); + testSimpleEval("select percent(386, 1000)", new String[]{"38.6"}); } } diff --git a/tajo-core/src/test/resources/python/test_funcs.py b/tajo-core/src/test/resources/python/test_funcs.py index c816232fc6..d6b7db5cd1 100644 --- a/tajo-core/src/test/resources/python/test_funcs.py +++ b/tajo-core/src/test/resources/python/test_funcs.py @@ -14,6 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +from tajo_util import outputType + @outputType('int4') def return_one(): return 1 diff --git a/tajo-core/src/test/resources/python/test_funcs2.py b/tajo-core/src/test/resources/python/test_funcs2.py index 1565d18469..e8db7b5d88 100644 --- a/tajo-core/src/test/resources/python/test_funcs2.py +++ b/tajo-core/src/test/resources/python/test_funcs2.py @@ -14,6 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. +from tajo_util import outputType + #Percent- Percentage @outputType("float8") def percent(num, total): diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java index c790ebabf6..bdacc369da 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java @@ -23,13 +23,10 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.tajo.OverridableConf; -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.*; +import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.function.FunctionInvocation; import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.function.PythonInvocationDesc; import org.apache.tajo.plan.function.python.JythonUtils; @@ -39,15 +36,16 @@ import org.apache.tajo.storage.VTuple; import java.io.*; +import java.util.Scanner; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; public class PythonFunctionInvoke2 extends FunctionInvoke { - private static final Log log = LogFactory.getLog(PythonFunctionInvoke2.class); + private static final Log LOG = LogFactory.getLog(PythonFunctionInvoke2.class); private static final String PYTHON_CONTROLLER_JAR_PATH = "/python/controller.py"; //Relative to root of tajo jar. - private static final String PYTHON_PIG_UTIL_PATH = "/python/tajo_util.py"; //Relative to root of tajo jar. + private static final String PYTHON_TAJO_UTIL_PATH = "/python/tajo_util.py"; //Relative to root of tajo jar. //Indexes for arguments being passed to external process private static final int UDF_LANGUAGE = 0; @@ -90,8 +88,12 @@ public class PythonFunctionInvoke2 extends FunctionInvoke { private PythonInvocationDesc invocationDesc; private Schema inSchema; private Schema outSchema; + private int [] projectionCols; private boolean isBinded = false; + private CSVLineSerDe lineSerDe = new CSVLineSerDe(); + private TableMeta pipeMeta; + public static final String TURN_ON_OUTPUT_CAPTURING = "TURN_ON_OUTPUT_CAPTURING"; public PythonFunctionInvoke2(FunctionDesc functionDesc) { @@ -110,6 +112,8 @@ public PythonFunctionInvoke2(FunctionDesc functionDesc) { inSchema.addColumn(new Column("in_" + i, paramTypes[i])); } outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); + projectionCols = new int[]{0}; + pipeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.TEXTFILE); } @Override @@ -125,7 +129,7 @@ public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTyp } private StreamingCommand startUdfController() throws IOException { - StreamingCommand sc = new StreamingCommand(constructCommand()); + StreamingCommand sc = new StreamingCommand(buildCommand()); ProcessBuilder processBuilder = StreamingUtil.createProcess(queryContext, sc); process = processBuilder.start(); @@ -133,7 +137,7 @@ private StreamingCommand startUdfController() throws IOException { return sc; } - private String[] constructCommand() throws IOException { + private String[] buildCommand() throws IOException { String[] command = new String[10]; // String jarPath = conf.get("mapreduce.job.jar"); @@ -148,7 +152,9 @@ private String[] constructCommand() throws IOException { // } // String standardOutputRootWriteLocation = soc.getStandardOutputRootWriteLocation(); - String standardOutputRootWriteLocation = System.getProperty("TAJO_LOG_DIR"); +// String standardOutputRootWriteLocation = System.getProperty("tajo.log.dir"); + // TODO + String standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; // if (execType.isLocal()) { @@ -169,20 +175,28 @@ private String[] constructCommand() throws IOException { command[UDF_LANGUAGE] = "python"; command[PATH_TO_CONTROLLER_FILE] = getControllerPath(); int lastSeparator = filePath.lastIndexOf(File.separator) + 1; - command[UDF_FILE_NAME] = filePath.substring(lastSeparator); + String fileName = filePath.substring(lastSeparator); + fileName = fileName.endsWith(".py") ? fileName.substring(0, fileName.length()-3) : fileName; + command[UDF_FILE_NAME] = fileName; command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; + // TODO String fileCachePath = filePath.substring(0, lastSeparator); - command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; +// command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; + command[PATH_TO_FILE_CACHE] = "'" + "/Users/jihoonson/Projects/tajo/tajo-core/src/test/resources/python/" + "'"; command[STD_OUT_OUTPUT_PATH] = outFileName; command[STD_ERR_OUTPUT_PATH] = errOutFileName; command[CONTROLLER_LOG_FILE_PATH] = controllerLogFileName; - command[IS_ILLUSTRATE] = ""; + command[IS_ILLUSTRATE] = "false"; // ensureUserFileAvailable(command, fileCachePath); + for (String cmd : command) { + LOG.info(cmd); + } + return command; } @@ -232,9 +246,11 @@ private String getUserFileExtension() { } private void createInputHandlers() { - RowStoreUtil.RowStoreEncoder serializer = RowStoreUtil.createEncoder(inSchema); + TextLineSerializer serializer = lineSerDe.createSerializer(inSchema, pipeMeta); + serializer.init(); this.inputHandler = new StreamingUDFInputHandler(serializer); - RowStoreUtil.RowStoreDecoder deserializer = RowStoreUtil.createDecoder(outSchema); + TextLineDeserializer deserializer = lineSerDe.createDeserializer(outSchema, pipeMeta, projectionCols); + deserializer.init(); this.outputHandler = new StreamingUDFOutputHandler(deserializer); } @@ -285,11 +301,11 @@ private String getControllerPath() throws IOException { pythonControllerStream.close(); } controllerFile.deleteOnExit(); - File pigUtilFile = new File(controllerFile.getParent() + "/pig_util.py"); - pigUtilFile.deleteOnExit(); - InputStream pythonUtilStream = this.getClass().getResourceAsStream(PYTHON_PIG_UTIL_PATH); + File tajoUtilFile = new File(controllerFile.getParent() + "/tajo_util.py"); + tajoUtilFile.deleteOnExit(); + InputStream pythonUtilStream = this.getClass().getResourceAsStream(PYTHON_TAJO_UTIL_PATH); try { - FileUtils.copyInputStreamToFile(pythonUtilStream, pigUtilFile); + FileUtils.copyInputStreamToFile(pythonUtilStream, tajoUtilFile); } finally { pythonUtilStream.close(); } @@ -320,13 +336,15 @@ private Datum getOutput(Tuple input) { // } try { - if (this.inSchema == null || this.inSchema.size() == 0) { +// if (this.inSchema == null || this.inSchema.size() == 0) { + if (input == null) { //When nothing is passed into the UDF the tuple //being sent is the full tuple for the relation. //We want it to be nothing (since that's what the user wrote). // input = TupleFactory.getInstance().newTuple(0); input = new VTuple(0); } + LOG.info("input: " + input); inputQueue.put(input); } catch (Exception e) { throw new RuntimeException("Failed adding input to inputQueue", e); @@ -351,7 +369,9 @@ private Datum getOutput(Tuple input) { throw new RuntimeException(outerrThreadsError); } - return JythonUtils.objectToDatum(outSchema.getColumn(0).getDataType(), o); +// Datum out = JythonUtils.objectToDatum(outSchema.getColumn(0).getDataType(), o); +// LOG.info("out: " + out); + return (Datum) o; } /** @@ -364,18 +384,20 @@ class ProcessInputThread extends Thread { public void run() { try { - log.debug("Starting PIT"); + LOG.info("Starting PIT"); while (true) { Tuple inputTuple = inputQueue.take(); + LOG.info("PIT: " + inputTuple); inputHandler.putNext(inputTuple); try { stdin.flush(); + LOG.info("PIT flushed"); } catch(Exception e) { return; } } } catch (Exception e) { - log.error(e); + LOG.error(e); } } } @@ -394,8 +416,8 @@ class ProcessOutputThread extends Thread { public void run() { Object o = null; try{ - log.debug("Starting POT"); - //StreamUDFToPig wraps object in single element tuple + LOG.info("Starting POT"); + o = outputHandler.getNext().get(0); while (o != OutputHandler.END_OF_OUTPUT) { if (o != null) @@ -423,7 +445,7 @@ public void run() { } outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. } catch(InterruptedException ie) { - log.error(ie); + LOG.error(ie); } } } @@ -437,7 +459,7 @@ public ProcessErrorThread() { public void run() { try { - log.debug("Starting PET"); + LOG.info("Starting PET"); Integer lineNumber = null; StringBuffer error = new StringBuffer(); String errInput; @@ -465,9 +487,9 @@ public void run() { stderr = null; } } catch (IOException e) { - log.debug("Process Ended", e); + LOG.info("Process Ended", e); } catch (Exception e) { - log.error("standard error problem", e); + LOG.error("standard error problem", e); } } } @@ -477,4 +499,38 @@ public void run() { process.destroy(); } } +// +// public static void main(String[] args) throws IOException { +// String line; +// Scanner scan = new Scanner(System.in); +// +// Process process = Runtime.getRuntime ().exec("/bin/bash"); +// OutputStream stdin = process.getOutputStream(); +// InputStream stderr = process.getErrorStream(); +// InputStream stdout = process.getInputStream(); +// +// BufferedReader reader = new BufferedReader (new InputStreamReader(stdout)); +// BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(stdin)); +// +// while (scan.hasNext()) { +// String input = scan.nextLine(); +// if (input.trim().equals("exit")) { +// // Putting 'exit' amongst the echo --EOF--s below doesn't work. +// writer.write("exit\n"); +// } else { +//// writer.write("((" + input + ") && echo --EOF--) || echo --EOF--\n"); +// writer.write("((" + input + ") && echo --EOF--)\n"); +// } +// writer.flush(); +// +// line = reader.readLine(); +// while (line != null && ! line.trim().equals("--EOF--")) { +// System.out.println ("Stdout: " + line); +// line = reader.readLine(); +// } +// if (line == null) { +// break; +// } +// } +// } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java index 7f5002f223..fb0e65286b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java @@ -98,24 +98,6 @@ private static void setupEnvironment(OverridableConf queryContext, ProcessBuilde env.put(PATH, envPath); } -// protected static void addJobConfToEnvironment(OverridableConf queryContext, Map env) { -// String propsToSend = queryContext.get(PIG_STREAMING_ENVIRONMENT); -// LOG.debug("Properties to ship to streaming environment set in "+PIG_STREAMING_ENVIRONMENT+": " + propsToSend); -// if (propsToSend == null) { -// return; -// } -// -// for (String prop : propsToSend.split(",")) { -// String value = conf.get(prop); -// if (value == null) { -// LOG.warn("Property set in "+PIG_STREAMING_ENVIRONMENT+" not found in Configuration: " + prop); -// continue; -// } -// LOG.debug("Setting property in streaming environment: " + prop); -// envPut(env, prop, value); -// } -// } - private static void envPut(Map env, String name, String value) { if (LOG.isDebugEnabled()) { LOG.debug("Add env entry:" + name + "=" + value); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index 179cbaf0ef..09284e0829 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -43,8 +43,6 @@ public class PythonScriptEngine extends TajoScriptEngine { public static Set registerFunctions(String path, String namespace) throws IOException { -// String command = pigContext.getProperties().getProperty( -// PigConfiguration.PIG_STREAMING_UDF_PYTHON_COMMAND, "python"); Set functionDescs = TUtil.newHashSet(); String command = "python"; @@ -67,17 +65,8 @@ public static Set registerFunctions(String path, String namespace) namespace = namespace == null ? "" : namespace + NAMESPACE_SEPARATOR; for(FuncInfo funcInfo : functions) { String alias = namespace + funcInfo.funcName; -// String execType = (pigContext.getExecType() == ExecType.LOCAL? "local" : "mapreduce"); -// String isIllustrate = (Boolean.valueOf(pigContext.inIllustrator)).toString(); log.debug("Registering Function: " + alias); -// pigContext.registerFunction(alias, -// new FuncSpec("StreamingUDF", -// new String[] { -// command, -// fileName, name, -// schemaString, schemaLineNumber, -// execType, isIllustrate -// })); + TajoDataTypes.DataType returnType = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.valueOf(funcInfo.returnType)); FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, funcInfo.funcName, returnType, createParamTypes(funcInfo.paramNum)); @@ -98,14 +87,6 @@ private static TajoDataTypes.DataType[] createParamTypes(int paramNum) { return paramTypes; } -// @Override -// protected Map> main(PigContext context, -// String scriptFile) throws IOException { -// log.warn("ScriptFile: " + scriptFile); -// registerFunctions(scriptFile, null, context); -// return getPigStatsMap(); -// } - @Override protected String getScriptingLang() { return "streaming_python"; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java index 50664b4eee..0db11058b8 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java @@ -49,77 +49,6 @@ public ScriptingOutputCapturer(OverridableConf queryContext, FunctionDesc functi this.functionDesc = functionDesc; } -// public String getStandardOutputRootWriteLocation() throws IOException { -// System.getProperty() -// String jobId = conf.get(MRConfiguration.JOB_ID); -// String taskId = conf.get(MRConfiguration.TASK_ID); -// String hadoopLogDir = System.getProperty("yarn.app.container.log.dir"); -// if (hadoopLogDir == null) { -// hadoopLogDir = conf.get("yarn.app.container.log.dir"); -// } -// if (hadoopLogDir == null) { -// hadoopLogDir = System.getProperty("hadoop.log.dir"); -// } -// if (hadoopLogDir == null) { -// hadoopLogDir = conf.get("hadoop.log.dir"); -// } -// -// String tmpDir = conf.get("hadoop.tmp.dir"); -// boolean fallbackToTmp = (hadoopLogDir == null); -// if (!fallbackToTmp) { -// try { -// if (!(new File(hadoopLogDir).canWrite())) { -// fallbackToTmp = true; -// } -// } -// catch (SecurityException e) { -// fallbackToTmp = true; -// } -// finally { -// if (fallbackToTmp) -// log.warn(String.format("Insufficient permission to write into %s. Change path to: %s", hadoopLogDir, tmpDir)); -// } -// } -// if (fallbackToTmp) { -// hadoopLogDir = tmpDir; -// } -// log.debug("JobId: " + jobId); -// log.debug("TaskId: " + taskId); -// log.debug("hadoopLogDir: " + hadoopLogDir); -// -// if (execType.isLocal() || conf.getBoolean(PigImplConstants.CONVERTED_TO_FETCH, false)) { -// String logDir = System.getProperty("pig.udf.scripting.log.dir"); -// if (logDir == null) -// logDir = "."; -// return logDir + "/" + (taskId == null ? "" : (taskId + "_")); -// } else { -// String taskLogDir = getTaskLogDir(jobId, taskId, hadoopLogDir); -// return taskLogDir + "/"; -// } -// return null; -// } - -// public String getTaskLogDir(String jobId, String taskId, String hadoopLogDir) throws IOException { -// String taskLogDir = null; -// String defaultUserLogDir = hadoopLogDir + File.separator + "userlogs"; -// -// if ( new File(defaultUserLogDir + File.separator + jobId).exists() ) { -// taskLogDir = defaultUserLogDir + File.separator + jobId + File.separator + taskId; -// } else if ( new File(defaultUserLogDir + File.separator + taskId).exists() ) { -// taskLogDir = defaultUserLogDir + File.separator + taskId; -// } else if ( new File(defaultUserLogDir).exists() ){ -// taskLogDir = defaultUserLogDir; -// } else { -// taskLogDir = hadoopLogDir + File.separator + "udfOutput"; -// File dir = new File(taskLogDir); -// dir.mkdirs(); -// if (!dir.exists()) { -// throw new IOException("Could not create directory: " + taskLogDir); -// } -// } -// return taskLogDir; -// } - public static void startCapturingOutput() { ScriptingOutputCapturer.captureOutput = true; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java index f306b360fb..277ae45198 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java @@ -87,20 +87,20 @@ private void fillBuffer() throws IOException { boolean release = true; try { int readBytes = tailBytes; - for (; ; ) { +// for (; ; ) { int localReadBytes = buffer.writeBytes(channel, this.bufferSize - readBytes); if (localReadBytes < 0) { if (buffer.isWritable()) { //if read bytes is less than the buffer capacity, there is no more bytes in the channel eof = true; } - break; +// break; } readBytes += localReadBytes; - if (readBytes == bufferSize) { - break; - } - } +// if (readBytes == bufferSize) { +// break; +// } +// } this.readBytes += (readBytes - tailBytes); release = false; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineDeserializer.java new file mode 100644 index 0000000000..1969d90d39 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineDeserializer.java @@ -0,0 +1,99 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufProcessor; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; + +public class CSVLineDeserializer extends TextLineDeserializer { + private ByteBufProcessor processor; + private FieldSerializerDeserializer fieldSerDer; + private ByteBuf nullChars; + private int delimiterCompensation; + + public CSVLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) { + super(schema, meta, targetColumnIndexes); + } + + @Override + public void init() { + byte[] delimiter = CSVLineSerDe.getFieldDelimiter(meta); + this.processor = new FieldSplitProcessor(delimiter[0]); + this.delimiterCompensation = delimiter.length - 1; + + if (nullChars != null) { + nullChars.release(); + } + nullChars = TextLineSerDe.getNullChars(meta); + + fieldSerDer = new TextFieldSerializerDeserializer(meta); + } + + public void deserialize(final ByteBuf lineBuf, Tuple output) throws IOException, TextLineParsingError { + int[] projection = targetColumnIndexes; + if (lineBuf == null || targetColumnIndexes == null || targetColumnIndexes.length == 0) { + return; + } + + final int rowLength = lineBuf.readableBytes(); + int start = 0, fieldLength = 0, end = 0; + + //Projection + int currentTarget = 0; + int currentIndex = 0; + + while (end != -1) { + end = lineBuf.forEachByte(start, rowLength - start, processor); + + if (end < 0) { + fieldLength = rowLength - start; + } else { + fieldLength = end - start - delimiterCompensation; + } + + if (projection.length > currentTarget && currentIndex == projection[currentTarget]) { + lineBuf.setIndex(start, start + fieldLength); + Datum datum = fieldSerDer.deserialize(lineBuf, schema.getColumn(currentIndex), currentIndex, nullChars); + output.put(currentIndex, datum); + currentTarget++; + } + + if (projection.length == currentTarget) { + break; + } + + start = end + 1; + currentIndex++; + } + } + + @Override + public void release() { + if (nullChars != null) { + nullChars.release(); + nullChars = null; + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerDe.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerDe.java new file mode 100644 index 0000000000..566e5c947f --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerDe.java @@ -0,0 +1,42 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.storage.StorageConstants; +import org.apache.tajo.util.Bytes; + +public class CSVLineSerDe extends TextLineSerDe { + @Override + public TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) { + return new CSVLineDeserializer(schema, meta, targetColumnIndexes); + } + + @Override + public TextLineSerializer createSerializer(Schema schema, TableMeta meta) { + return new CSVLineSerializer(schema, meta); + } + + public static byte[] getFieldDelimiter(TableMeta meta) { + return StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_DELIMITER, + StorageConstants.DEFAULT_FIELD_DELIMITER)).getBytes(Bytes.UTF8_CHARSET); + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java new file mode 100644 index 0000000000..bd7127a7d0 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.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.tajo.plan.function.stream; + +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.AnyDatum; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; +import java.io.OutputStream; + +public class CSVLineSerializer extends TextLineSerializer { + private FieldSerializerDeserializer serde; + + private byte[] nullChars; + private byte[] delimiter; + private int columnNum; + + private final static String PARAM_DELIM = "|\t_"; + + public CSVLineSerializer(Schema schema, TableMeta meta) { + super(schema, meta); + } + + @Override + public void init() { + nullChars = TextLineSerDe.getNullCharsAsBytes(meta); + delimiter = ",".getBytes(); + columnNum = schema.size(); + + serde = new TextFieldSerializerDeserializer(meta); + } + + @Override + public int serialize(OutputStream out, Tuple input) throws IOException { + int writtenBytes = 0; + + for (int i = 0; i < columnNum; i++) { + Datum datum = input.get(i); + String typeStr; + if (datum.type() == TajoDataTypes.Type.ANY) { + typeStr = getTypeString(((AnyDatum)datum).getActual()); + } else { + typeStr = getTypeString(datum); + } + out.write(typeStr.getBytes()); + out.write(PARAM_DELIM.getBytes()); + + writtenBytes += serde.serialize(out, datum, schema.getColumn(i), i, nullChars); + + if (columnNum - 1 > i) { + out.write(delimiter); + writtenBytes += delimiter.length; + } + } + + return writtenBytes; + } + + @Override + public void release() { + } + + private static String getTypeString(Datum val) { + switch (val.type()) { + case NULL_TYPE: + return "-"; + case BOOLEAN: + return "B"; + case INT1: + case INT2: + case INT4: + return "I"; + case INT8: + return "L"; + case FLOAT4: + return "F"; + case FLOAT8: + return "D"; + case NUMERIC: + return "E"; + case CHAR: + case TEXT: + return "C"; + case DATE: + case TIME: + case TIMESTAMP: + return "T"; + case BLOB: + case INET4: + case INET6: + return "A"; + default: + throw new UnsupportedException(val.type().name()); + } + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSerializerDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSerializerDeserializer.java new file mode 100644 index 0000000000..29a5aa2131 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSerializerDeserializer.java @@ -0,0 +1,36 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBuf; +import org.apache.tajo.catalog.Column; +import org.apache.tajo.datum.Datum; + +import java.io.IOException; +import java.io.OutputStream; + + +public interface FieldSerializerDeserializer { + + int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) throws IOException; + + Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars) + throws IOException, TextLineParsingError; + +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSplitProcessor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSplitProcessor.java new file mode 100644 index 0000000000..2e89d397ee --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/FieldSplitProcessor.java @@ -0,0 +1,34 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBufProcessor; + +public class FieldSplitProcessor implements ByteBufProcessor { + private byte delimiter; //the ascii separate character + + public FieldSplitProcessor(byte recordDelimiterByte) { + this.delimiter = recordDelimiterByte; + } + + @Override + public boolean process(byte value) throws Exception { + return delimiter != value; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java index 373d083293..2681105adb 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -25,13 +25,17 @@ /** * {@link InputHandler} is responsible for handling the input to the - * Pig-Streaming external command. + * Tajo-Streaming external command. * * The managed executable could be fed input in a {@link InputType#SYNCHRONOUS} * manner via its stdin or in an {@link InputType#ASYNCHRONOUS} * manner via an external file which is subsequently read by the executable. */ public abstract class InputHandler { + + private final static byte[] END_OF_RECORD_DELIM = "|_\n".getBytes(); + private final static byte[] END_OF_STREAM = ("C" + "\\x04" + "|_\n").getBytes(); + /** * */ @@ -43,7 +47,7 @@ public enum InputType {SYNCHRONOUS, ASYNCHRONOUS} * It is the responsibility of the concrete sub-classes to setup and * manage the serializer. */ - protected RowStoreUtil.RowStoreEncoder serializer; + protected TextLineSerializer serializer; private OutputStream out; @@ -63,7 +67,8 @@ public enum InputType {SYNCHRONOUS, ASYNCHRONOUS} * @throws IOException */ public void putNext(Tuple t) throws IOException { - out.write(serializer.toBytes(t)); + serializer.serialize(out, t); + out.write(END_OF_RECORD_DELIM); } /** diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java index 83bab23544..ac5813a18b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java @@ -19,15 +19,20 @@ package org.apache.tajo.plan.function.stream; import com.google.common.base.Charsets; +import io.netty.buffer.ByteBuf; import org.apache.tajo.OverridableConf; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.datum.NullDatum; import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; import java.io.IOException; import java.io.InputStream; /** * {@link OutputHandler} is responsible for handling the output of the - * Pig-Streaming external command. + * Tajo-Streaming external command. * * The output of the managed executable could be fetched in a * {@link OutputType#SYNCHRONOUS} manner via its stdout or in an @@ -35,25 +40,25 @@ * process wrote its output. */ public abstract class OutputHandler { + private static int DEFAULT_BUFFER = 64 * 1024; public static final Object END_OF_OUTPUT = new Object(); private static final byte[] DEFAULT_RECORD_DELIM = new byte[] {'\n'}; public enum OutputType {SYNCHRONOUS, ASYNCHRONOUS} - protected RowStoreUtil.RowStoreDecoder deserializer; + protected TextLineDeserializer deserializer; protected ByteBufLineReader in = null; -// private Text currValue = new Text(); private String currValue = null; -// private BufferedPositionedInputStream istream; private InputStream istream; //Both of these ignore the trailing \n. So if the //default delimiter is "\n" recordDelimStr is "". private String recordDelimStr = null; private int recordDelimLength = 0; + private Tuple tuple = new VTuple(1); /** * Get the handled OutputType. @@ -76,6 +81,8 @@ public void bindTo(String fileName, InputStream is, long offset, long end) throws IOException { this.istream = is; this.in = new ByteBufLineReader(new ByteBufInputChannel(istream)); + + // TODO } /** @@ -93,9 +100,14 @@ public Tuple getNext() throws IOException { if (!readValue()) { return null; } - byte[] newBytes = new byte[currValue.length()]; - System.arraycopy(currValue.getBytes(), 0, newBytes, 0, currValue.length()); - return deserializer.toTuple(newBytes); + ByteBuf buf = BufferPool.directBuffer(DEFAULT_BUFFER); + buf.writeBytes(currValue.getBytes()); + try { + deserializer.deserialize(buf, tuple); + } catch (TextLineParsingError textLineParsingError) { + throw new IOException(textLineParsingError); + } + return tuple; } private boolean readValue() throws IOException { @@ -118,6 +130,11 @@ private boolean readValue() throws IOException { currValue += new String(lineBytes); } + if (currValue.contains("|_")) { + int pos = currValue.lastIndexOf("|_"); + currValue = currValue.substring(0, pos); + } + return true; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java deleted file mode 100644 index 44906cd0f6..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/RowStoreUtil.java +++ /dev/null @@ -1,290 +0,0 @@ -/* - * Lisensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.stream; - -import org.apache.tajo.catalog.Column; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.datum.IntervalDatum; -import org.apache.tajo.exception.UnknownDataTypeException; -import org.apache.tajo.exception.UnsupportedException; -import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; -import org.apache.tajo.util.BitArray; - -import java.nio.ByteBuffer; - -public class RowStoreUtil { - - public static int[] getTargetIds(Schema inSchema, Schema outSchema) { - int[] targetIds = new int[outSchema.size()]; - int i = 0; - for (Column target : outSchema.getColumns()) { - targetIds[i] = inSchema.getColumnId(target.getQualifiedName()); - i++; - } - - return targetIds; - } - - public static Tuple project(Tuple in, Tuple out, int[] targetIds) { - out.clear(); - for (int idx = 0; idx < targetIds.length; idx++) { - out.put(idx, in.get(targetIds[idx])); - } - return out; - } - - public static RowStoreEncoder createEncoder(Schema schema) { - return new RowStoreEncoder(schema); - } - - public static RowStoreDecoder createDecoder(Schema schema) { - return new RowStoreDecoder(schema); - } - - public static class RowStoreDecoder { - - private Schema schema; - private BitArray nullFlags; - private int headerSize; - - private RowStoreDecoder(Schema schema) { - this.schema = schema; - nullFlags = new BitArray(schema.size()); - headerSize = nullFlags.bytesLength(); - } - - - public Tuple toTuple(byte [] bytes) { - nullFlags.clear(); - ByteBuffer bb = ByteBuffer.wrap(bytes); - Tuple tuple = new VTuple(schema.size()); - Column col; - TajoDataTypes.DataType type; - - bb.limit(headerSize); - nullFlags.fromByteBuffer(bb); - bb.limit(bytes.length); - - for (int i =0; i < schema.size(); i++) { - if (nullFlags.get(i)) { - tuple.put(i, DatumFactory.createNullDatum()); - continue; - } - - col = schema.getColumn(i); - type = col.getDataType(); - switch (type.getType()) { - case BOOLEAN: tuple.put(i, DatumFactory.createBool(bb.get())); break; - case BIT: - byte b = bb.get(); - tuple.put(i, DatumFactory.createBit(b)); - break; - - case CHAR: - byte c = bb.get(); - tuple.put(i, DatumFactory.createChar(c)); - break; - - case INT2: - short s = bb.getShort(); - tuple.put(i, DatumFactory.createInt2(s)); - break; - - case INT4: - case DATE: - int i_ = bb.getInt(); - tuple.put(i, DatumFactory.createFromInt4(type, i_)); - break; - - case INT8: - case TIME: - case TIMESTAMP: - long l = bb.getLong(); - tuple.put(i, DatumFactory.createFromInt8(type, l)); - break; - - case INTERVAL: - int month = bb.getInt(); - long milliseconds = bb.getLong(); - tuple.put(i, new IntervalDatum(month, milliseconds)); - break; - - case FLOAT4: - float f = bb.getFloat(); - tuple.put(i, DatumFactory.createFloat4(f)); - break; - - case FLOAT8: - double d = bb.getDouble(); - tuple.put(i, DatumFactory.createFloat8(d)); - break; - - case TEXT: - byte [] _string = new byte[bb.getInt()]; - bb.get(_string); - tuple.put(i, DatumFactory.createText(_string)); - break; - - case BLOB: - byte [] _bytes = new byte[bb.getInt()]; - bb.get(_bytes); - tuple.put(i, DatumFactory.createBlob(_bytes)); - break; - - case INET4: - byte [] _ipv4 = new byte[4]; - bb.get(_ipv4); - tuple.put(i, DatumFactory.createInet4(_ipv4)); - break; - case INET6: - // TODO - to be implemented - throw new UnsupportedException(type.getType().name()); - default: - throw new RuntimeException(new UnknownDataTypeException(type.getType().name())); - } - } - return tuple; - } - - public Schema getSchema() { - return schema; - } - } - - public static class RowStoreEncoder { - private Schema schema; - private BitArray nullFlags; - private int headerSize; - - private RowStoreEncoder(Schema schema) { - this.schema = schema; - nullFlags = new BitArray(schema.size()); - headerSize = nullFlags.bytesLength(); - } - public byte [] toBytes(Tuple tuple) { - nullFlags.clear(); - int size = estimateTupleDataSize(tuple); - ByteBuffer bb = ByteBuffer.allocate(size + headerSize); - bb.position(headerSize); - Column col; - for (int i = 0; i < schema.size(); i++) { - if (tuple.isNull(i)) { - nullFlags.set(i); - continue; - } - - col = schema.getColumn(i); - switch (col.getDataType().getType()) { - case NULL_TYPE: nullFlags.set(i); break; - case BOOLEAN: bb.put(tuple.get(i).asByte()); break; - case BIT: bb.put(tuple.get(i).asByte()); break; - case CHAR: bb.put(tuple.get(i).asByte()); break; - case INT2: bb.putShort(tuple.get(i).asInt2()); break; - case INT4: bb.putInt(tuple.get(i).asInt4()); break; - case INT8: bb.putLong(tuple.get(i).asInt8()); break; - case FLOAT4: bb.putFloat(tuple.get(i).asFloat4()); break; - case FLOAT8: bb.putDouble(tuple.get(i).asFloat8()); break; - case TEXT: - byte [] _string = tuple.get(i).asByteArray(); - bb.putInt(_string.length); - bb.put(_string); - break; - case DATE: bb.putInt(tuple.get(i).asInt4()); break; - case TIME: - case TIMESTAMP: - bb.putLong(tuple.get(i).asInt8()); - break; - case INTERVAL: - IntervalDatum interval = (IntervalDatum) tuple.get(i); - bb.putInt(interval.getMonths()); - bb.putLong(interval.getMilliSeconds()); - break; - case BLOB: - byte [] bytes = tuple.get(i).asByteArray(); - bb.putInt(bytes.length); - bb.put(bytes); - break; - case INET4: - byte [] ipBytes = tuple.get(i).asByteArray(); - bb.put(ipBytes); - break; - case INET6: bb.put(tuple.get(i).asByteArray()); break; - default: - throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name())); - } - } - - byte[] flags = nullFlags.toArray(); - int finalPosition = bb.position(); - bb.position(0); - bb.put(flags); - - bb.position(finalPosition); - bb.flip(); - byte [] buf = new byte [bb.limit()]; - bb.get(buf); - return buf; - } - - // Note that, NULL values are treated separately - private int estimateTupleDataSize(Tuple tuple) { - int size = 0; - Column col; - - for (int i = 0; i < schema.size(); i++) { - if (tuple.isNull(i)) { - continue; - } - - col = schema.getColumn(i); - switch (col.getDataType().getType()) { - case BOOLEAN: - case BIT: - case CHAR: size += 1; break; - case INT2: size += 2; break; - case DATE: - case INT4: - case FLOAT4: size += 4; break; - case TIME: - case TIMESTAMP: - case INT8: - case FLOAT8: size += 8; break; - case INTERVAL: size += 12; break; - case TEXT: - case BLOB: size += (4 + tuple.get(i).asByteArray().length); break; - case INET4: - case INET6: size += tuple.get(i).asByteArray().length; break; - default: - throw new RuntimeException(new UnknownDataTypeException(col.getDataType().getType().name())); - } - } - - size += 100; // optimistic reservation - - return size; - } - - public Schema getSchema() { - return schema; - } - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java index 62cd168219..dcc3c25835 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java @@ -22,7 +22,7 @@ public class StreamingUDFInputHandler extends InputHandler { - public StreamingUDFInputHandler(RowStoreUtil.RowStoreEncoder serializer) { + public StreamingUDFInputHandler(TextLineSerializer serializer) { this.serializer = serializer; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java index 122b035a24..75eb3b151b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java @@ -20,7 +20,7 @@ public class StreamingUDFOutputHandler extends OutputHandler { - public StreamingUDFOutputHandler(RowStoreUtil.RowStoreDecoder deserializer) { + public StreamingUDFOutputHandler(TextLineDeserializer deserializer) { this.deserializer = deserializer; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java new file mode 100644 index 0000000000..b809820877 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java @@ -0,0 +1,258 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import com.google.protobuf.Message; +import io.netty.buffer.ByteBuf; +import io.netty.util.CharsetUtil; +import org.apache.commons.codec.binary.Base64; +import org.apache.tajo.TajoConstants; +import org.apache.tajo.catalog.Column; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.datum.*; +import org.apache.tajo.datum.protobuf.ProtobufJsonFormat; +import org.apache.tajo.exception.UnsupportedException; +import org.apache.tajo.storage.StorageConstants; +import org.apache.tajo.util.NumberUtil; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.charset.CharsetDecoder; +import java.util.TimeZone; + +public class TextFieldSerializerDeserializer implements FieldSerializerDeserializer { + public static final byte[] trueBytes = "true".getBytes(); + public static final byte[] falseBytes = "false".getBytes(); + private static ProtobufJsonFormat protobufJsonFormat = ProtobufJsonFormat.getInstance(); + private final CharsetDecoder decoder = CharsetUtil.getDecoder(CharsetUtil.UTF_8); + + private final boolean hasTimezone; + private final TimeZone timezone; + + public TextFieldSerializerDeserializer(TableMeta meta) { + hasTimezone = meta.containsOption(StorageConstants.TIMEZONE); + timezone = TimeZone.getTimeZone(meta.getOption(StorageConstants.TIMEZONE, TajoConstants.DEFAULT_SYSTEM_TIMEZONE)); + } + + private static boolean isNull(ByteBuf val, ByteBuf nullBytes) { + return !val.isReadable() || nullBytes.equals(val); + } + + private static boolean isNullText(ByteBuf val, ByteBuf nullBytes) { + return val.readableBytes() > 0 && nullBytes.equals(val); + } + + @Override + public int serialize(OutputStream out, Datum datum, Column col, int columnIndex, byte[] nullChars) + throws IOException { + byte[] bytes; + int length = 0; + TajoDataTypes.DataType dataType = col.getDataType(); + + if (datum == null || datum instanceof NullDatum) { + switch (dataType.getType()) { + case CHAR: + case TEXT: + length = nullChars.length; + out.write(nullChars); + break; + default: + break; + } + return length; + } + + switch (dataType.getType()) { + case BOOLEAN: + out.write(datum.asBool() ? trueBytes : falseBytes); + length = trueBytes.length; + break; + case CHAR: + byte[] pad = new byte[dataType.getLength() - datum.size()]; + bytes = datum.asTextBytes(); + out.write(bytes); + out.write(pad); + length = bytes.length + pad.length; + break; + case TEXT: + case BIT: + case INT2: + case INT4: + case INT8: + case FLOAT4: + case FLOAT8: + case INET4: + case DATE: + case INTERVAL: + bytes = datum.asTextBytes(); + length = bytes.length; + out.write(bytes); + break; + case TIME: + if (hasTimezone) { + bytes = ((TimeDatum) datum).asChars(timezone, true).getBytes(); + } else { + bytes = datum.asTextBytes(); + } + length = bytes.length; + out.write(bytes); + break; + case TIMESTAMP: + if (hasTimezone) { + bytes = ((TimestampDatum) datum).asChars(timezone, true).getBytes(); + } else { + bytes = datum.asTextBytes(); + } + length = bytes.length; + out.write(bytes); + break; + case INET6: + case BLOB: + bytes = Base64.encodeBase64(datum.asByteArray(), false); + length = bytes.length; + out.write(bytes, 0, length); + break; + case PROTOBUF: + ProtobufDatum protobuf = (ProtobufDatum) datum; + byte[] protoBytes = protobufJsonFormat.printToString(protobuf.get()).getBytes(); + length = protoBytes.length; + out.write(protoBytes, 0, protoBytes.length); + break; + case NULL_TYPE: + break; + case ANY: + AnyDatum anyDatum = (AnyDatum) datum; + length = serialize(out, anyDatum.getActual(), new Column("any", anyDatum.getActual().type()), 0, nullChars); + break; + default: + throw new UnsupportedException(dataType.getType().name()); + } + return length; + } + + @Override + public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullChars) throws IOException { + Datum datum; + TajoDataTypes.Type type = col.getDataType().getType(); + boolean nullField; + if (type == TajoDataTypes.Type.TEXT || type == TajoDataTypes.Type.CHAR) { + nullField = isNullText(buf, nullChars); + } else { + nullField = isNull(buf, nullChars); + } + + if (nullField) { + datum = NullDatum.get(); + } else { + switch (type) { + case BOOLEAN: + byte bool = buf.readByte(); + datum = DatumFactory.createBool(bool == 't' || bool == 'T'); + break; + case BIT: + datum = DatumFactory.createBit(Byte.parseByte( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString())); + break; + case CHAR: + datum = DatumFactory.createChar( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString().trim()); + break; + case INT1: + case INT2: + datum = DatumFactory.createInt2((short) NumberUtil.parseInt(buf)); + break; + case INT4: + datum = DatumFactory.createInt4(NumberUtil.parseInt(buf)); + break; + case INT8: + datum = DatumFactory.createInt8(NumberUtil.parseLong(buf)); + break; + case FLOAT4: + datum = DatumFactory.createFloat4( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()); + break; + case FLOAT8: + datum = DatumFactory.createFloat8(NumberUtil.parseDouble(buf)); + break; + case TEXT: { + byte[] bytes = new byte[buf.readableBytes()]; + buf.readBytes(bytes); + datum = DatumFactory.createText(bytes); + break; + } + case DATE: + datum = DatumFactory.createDate( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()); + break; + case TIME: + if (hasTimezone) { + datum = DatumFactory.createTime( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString(), timezone); + } else { + datum = DatumFactory.createTime( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()); + } + break; + case TIMESTAMP: + if (hasTimezone) { + datum = DatumFactory.createTimestamp( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString(), timezone); + } else { + datum = DatumFactory.createTimestamp( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()); + } + break; + case INTERVAL: + datum = DatumFactory.createInterval( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()); + break; + case PROTOBUF: { + ProtobufDatumFactory factory = ProtobufDatumFactory.get(col.getDataType()); + Message.Builder builder = factory.newBuilder(); + try { + byte[] bytes = new byte[buf.readableBytes()]; + buf.readBytes(bytes); + protobufJsonFormat.merge(bytes, builder); + datum = factory.createDatum(builder.build()); + } catch (IOException e) { + e.printStackTrace(); + throw new RuntimeException(e); + } + break; + } + case INET4: + datum = DatumFactory.createInet4( + decoder.decode(buf.nioBuffer(buf.readerIndex(), buf.readableBytes())).toString()); + break; + case BLOB: { + byte[] bytes = new byte[buf.readableBytes()]; + buf.readBytes(bytes); +// datum = DatumFactory.createBlob(Base64.decodeBase64(bytes)); + datum = DatumFactory.createBlob(bytes); + break; + } + default: + datum = NullDatum.get(); + break; + } + } + return datum; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineDeserializer.java new file mode 100644 index 0000000000..c2ea30ccdc --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineDeserializer.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBuf; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; + +/** + * Reads a text line and fills a Tuple with values + */ +public abstract class TextLineDeserializer { + protected final Schema schema; + protected final TableMeta meta; + protected final int[] targetColumnIndexes; + + public TextLineDeserializer(Schema schema, TableMeta meta, int[] targetColumnIndexes) { + this.schema = schema; + this.meta = meta; + this.targetColumnIndexes = targetColumnIndexes; + } + + /** + * Initialize SerDe + */ + public abstract void init(); + + /** + * It fills a tuple with a read fields in a given line. + * + * @param buf Read line + * @param output Tuple to be filled with read fields + * @throws IOException + */ + public abstract void deserialize(final ByteBuf buf, Tuple output) throws IOException, TextLineParsingError; + + /** + * Release external resources + */ + public abstract void release(); +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineParsingError.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineParsingError.java new file mode 100644 index 0000000000..9048e27c16 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineParsingError.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +public class TextLineParsingError extends Exception { + + public TextLineParsingError(Throwable t) { + super(t); + } + + public TextLineParsingError(String message, Throwable t) { + super(t.getMessage() + ", Error line: " + message); + } + +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerDe.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerDe.java new file mode 100644 index 0000000000..89b169c0ee --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerDe.java @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang.StringEscapeUtils; +import org.apache.commons.lang.StringUtils; +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.datum.NullDatum; +import org.apache.tajo.storage.StorageConstants; +import org.apache.tajo.util.Bytes; + +/** + * Pluggable Text Line SerDe class + */ +public abstract class TextLineSerDe { + + public TextLineSerDe() { + } + + public abstract TextLineDeserializer createDeserializer(Schema schema, TableMeta meta, int [] targetColumnIndexes); + + public abstract TextLineSerializer createSerializer(Schema schema, TableMeta meta); + + public static ByteBuf getNullChars(TableMeta meta) { + byte[] nullCharByteArray = getNullCharsAsBytes(meta); + + ByteBuf nullChars = BufferPool.directBuffer(nullCharByteArray.length, nullCharByteArray.length); + nullChars.writeBytes(nullCharByteArray); + + return nullChars; + } + + public static byte [] getNullCharsAsBytes(TableMeta meta) { + byte [] nullChars; + + String nullCharacters = StringEscapeUtils.unescapeJava(meta.getOption(StorageConstants.TEXT_NULL, + NullDatum.DEFAULT_TEXT)); + if (StringUtils.isEmpty(nullCharacters)) { + nullChars = NullDatum.get().asTextBytes(); + } else { + nullChars = nullCharacters.getBytes(Bytes.UTF8_CHARSET); + } + + return nullChars; + } + +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerializer.java new file mode 100644 index 0000000000..ea8576c2f1 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextLineSerializer.java @@ -0,0 +1,45 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function.stream; + +import org.apache.tajo.catalog.Schema; +import org.apache.tajo.catalog.TableMeta; +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; +import java.io.OutputStream; + +/** + * Write a Tuple into single text formatted line + */ +public abstract class TextLineSerializer { + protected Schema schema; + protected TableMeta meta; + + public TextLineSerializer(Schema schema, TableMeta meta) { + this.schema = schema; + this.meta = meta; + } + + public abstract void init(); + + public abstract int serialize(OutputStream out, Tuple input) throws IOException; + + public abstract void release(); +} From 6d0511193eaa6f1e1ee64aa85ce8a36b5bfdfeed Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 6 Apr 2015 11:42:14 +0900 Subject: [PATCH 27/55] TAJO-1344 --- .../main/java/org/apache/tajo/master/GlobalEngine.java | 2 -- .../apache/tajo/engine/function/TestPythonFunctions.java | 8 ++++---- .../org/apache/tajo/plan/expr/PythonFunctionInvoke2.java | 6 ++++-- .../tajo/plan/function/stream/CSVLineSerializer.java | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java index 9d853a5ba9..4e87d35c38 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java @@ -38,7 +38,6 @@ import org.apache.tajo.master.TajoMaster.MasterContext; import org.apache.tajo.master.exec.DDLExecutor; import org.apache.tajo.master.exec.QueryExecutor; -import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager; import org.apache.tajo.session.Session; import org.apache.tajo.plan.*; import org.apache.tajo.plan.logical.InsertNode; @@ -70,7 +69,6 @@ public class GlobalEngine extends AbstractService { private LogicalPlanner planner; private LogicalOptimizer optimizer; private LogicalPlanVerifier annotatedPlanVerifier; - private DistributedQueryHookManager hookManager; private QueryExecutor queryExecutor; private DDLExecutor ddlExecutor; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java index 668cfadc54..09e26b73a9 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -27,12 +27,12 @@ public class TestPythonFunctions extends ExprTestBase { @Test public void test() throws IOException { -// testSimpleEval("select return_one()", new String[]{"1"}); -// testSimpleEval("select helloworld()", new String[]{"Hello, World"}); - testSimpleEval("select sum_py(1,2)", new String[]{"3"}); + testSimpleEval("select return_one()", new String[]{"1"}); + testSimpleEval("select helloworld()", new String[]{"Hello, World"}); testSimpleEval("select concat_py('1')", new String[]{"11"}); testSimpleEval("select comma_format(12345)", new String[]{"12,345"}); - testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); + testSimpleEval("select sum_py(1,2)", new String[]{"3"}); testSimpleEval("select percent(386, 1000)", new String[]{"38.6"}); + testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java index bdacc369da..70b68a8dbd 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java @@ -154,7 +154,8 @@ private String[] buildCommand() throws IOException { // String standardOutputRootWriteLocation = soc.getStandardOutputRootWriteLocation(); // String standardOutputRootWriteLocation = System.getProperty("tajo.log.dir"); // TODO - String standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; +// String standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; + String standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; // if (execType.isLocal()) { @@ -185,7 +186,8 @@ private String[] buildCommand() throws IOException { // TODO String fileCachePath = filePath.substring(0, lastSeparator); // command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; - command[PATH_TO_FILE_CACHE] = "'" + "/Users/jihoonson/Projects/tajo/tajo-core/src/test/resources/python/" + "'"; +// command[PATH_TO_FILE_CACHE] = "'" + "/Users/jihoonson/Projects/tajo/tajo-core/src/test/resources/python/" + "'"; + command[PATH_TO_FILE_CACHE] = "'" + "/home/jihoon/Projects/tajo/tajo-core/src/test/resources/python/" + "'"; command[STD_OUT_OUTPUT_PATH] = outFileName; command[STD_ERR_OUTPUT_PATH] = errOutFileName; command[CONTROLLER_LOG_FILE_PATH] = controllerLogFileName; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java index bd7127a7d0..10ae9eab47 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java @@ -45,7 +45,7 @@ public CSVLineSerializer(Schema schema, TableMeta meta) { @Override public void init() { nullChars = TextLineSerDe.getNullCharsAsBytes(meta); - delimiter = ",".getBytes(); + delimiter = "|,_".getBytes(); columnNum = schema.size(); serde = new TextFieldSerializerDeserializer(meta); From 9c0fb84a9020ffa4316d00d084cf72c07ec31889 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Mon, 6 Apr 2015 16:08:07 +0900 Subject: [PATCH 28/55] All tests passed --- .../java/org/apache/tajo/datum/BlobDatum.java | 2 +- .../org/apache/tajo/datum/DatumFactory.java | 3 +- tajo-plan/pom.xml | 5 - .../plan/function/python/JythonUtils.java | 212 +++++++++--------- .../apache/tajo/storage/TestLazyTuple.java | 2 +- .../tajo/storage/TestDelimitedTextFile.java | 2 +- .../tajo/storage/json/TestJsonSerDe.java | 2 +- 7 files changed, 112 insertions(+), 116 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java index 2fc923fdd6..ef1323e333 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java @@ -132,7 +132,7 @@ public boolean equals(Object obj) { initFromBytes(); other.initFromBytes(); // return bb.equals(other.bb); - return Arrays.equals(bb.array(), other.bb.array()); + return Arrays.equals(this.val, other.val); } return false; diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java index 20a2d8962d..0032b5c95d 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java @@ -19,6 +19,7 @@ package org.apache.tajo.datum; import com.google.protobuf.Message; +import org.apache.commons.codec.binary.Base64; import org.apache.tajo.common.TajoDataTypes.DataType; import org.apache.tajo.common.TajoDataTypes.Type; import org.apache.tajo.exception.InvalidCastException; @@ -386,7 +387,7 @@ public static BlobDatum createBlob(byte[] val, int offset, int length) { } public static BlobDatum createBlob(String val) { - return new BlobDatum(val.getBytes()); + return new BlobDatum(Base64.encodeBase64(val.getBytes())); } public static Inet4Datum createInet4(int encoded) { diff --git a/tajo-plan/pom.xml b/tajo-plan/pom.xml index a7789c0113..f2f00671aa 100644 --- a/tajo-plan/pom.xml +++ b/tajo-plan/pom.xml @@ -185,11 +185,6 @@ mockito-core test - - org.python - jython-standalone - 2.7-b1 - diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java index 496a3cb1f3..fdeee6560d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java @@ -26,69 +26,69 @@ import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.storage.Tuple; -import org.python.core.*; +//import org.python.core.*; public class JythonUtils { - /** - * Convert a datum to a PyObject. - * @param v - * @return - */ - public static PyObject datumToPyObject(Datum v) { - Preconditions.checkArgument(v.type() == TajoDataTypes.Type.ANY); - Datum actual = ((AnyDatum) v).getActual(); - switch (actual.type()) { - case NULL_TYPE: - return Py.java2py(null); - case BOOLEAN: - return Py.java2py(actual.asBool()); - case UINT1: - case INT1: - return Py.java2py(actual.asInt2()); - case UINT2: - case INT2: - return Py.java2py(actual.asInt2()); - case UINT4: - case INT4: - return Py.java2py(actual.asInt4()); - case UINT8: - case INT8: - return Py.java2py(actual.asInt8()); - case FLOAT4: - case FLOAT8: - return Py.java2py(actual.asFloat8()); - case CHAR: - case VARCHAR: - case TEXT: - return Py.java2py(actual.asChars()); - case NCHAR: - case NVARCHAR: - return Py.java2py(actual.asUnicodeChars()); - case BLOB: - return Py.java2py(actual.asByteArray()); - case INET4: - return Py.java2py(actual.asByteArray()); - case INET6: - return Py.java2py(actual.asByteArray()); - default: - throw new UnsupportedException("Unsupported type: " + actual.type()); - } - } - - /** - * Convert a Tajo tuple to a PyTuple - * @param tuple - * @return - */ - public static PyTuple tupleToPyTuple(Tuple tuple) { - PyObject[] pyTuple = new PyObject[tuple.size()]; - int i = 0; - for (Datum v : tuple.getValues()) { - pyTuple[i++] = datumToPyObject(v); - } - return new PyTuple(pyTuple); - } +// /** +// * Convert a datum to a PyObject. +// * @param v +// * @return +// */ +// public static PyObject datumToPyObject(Datum v) { +// Preconditions.checkArgument(v.type() == TajoDataTypes.Type.ANY); +// Datum actual = ((AnyDatum) v).getActual(); +// switch (actual.type()) { +// case NULL_TYPE: +// return Py.java2py(null); +// case BOOLEAN: +// return Py.java2py(actual.asBool()); +// case UINT1: +// case INT1: +// return Py.java2py(actual.asInt2()); +// case UINT2: +// case INT2: +// return Py.java2py(actual.asInt2()); +// case UINT4: +// case INT4: +// return Py.java2py(actual.asInt4()); +// case UINT8: +// case INT8: +// return Py.java2py(actual.asInt8()); +// case FLOAT4: +// case FLOAT8: +// return Py.java2py(actual.asFloat8()); +// case CHAR: +// case VARCHAR: +// case TEXT: +// return Py.java2py(actual.asChars()); +// case NCHAR: +// case NVARCHAR: +// return Py.java2py(actual.asUnicodeChars()); +// case BLOB: +// return Py.java2py(actual.asByteArray()); +// case INET4: +// return Py.java2py(actual.asByteArray()); +// case INET6: +// return Py.java2py(actual.asByteArray()); +// default: +// throw new UnsupportedException("Unsupported type: " + actual.type()); +// } +// } +// +// /** +// * Convert a Tajo tuple to a PyTuple +// * @param tuple +// * @return +// */ +// public static PyTuple tupleToPyTuple(Tuple tuple) { +// PyObject[] pyTuple = new PyObject[tuple.size()]; +// int i = 0; +// for (Datum v : tuple.getValues()) { +// pyTuple[i++] = datumToPyObject(v); +// } +// return new PyTuple(pyTuple); +// } public static Datum objectToDatum(TajoDataTypes.DataType type, Object o) { switch (type.getType()) { @@ -133,52 +133,52 @@ public static Datum objectToDatum(TajoDataTypes.DataType type, Object o) { } } - /** - * Convert a PyObject to a datum. - * @param object - * @return - */ - public static Datum pyObjectToDatum(PyObject object) { - if (object instanceof PyLong) { - return DatumFactory.createInt8((Long) object.__tojava__(Long.class)); - } else if (object instanceof PyBoolean) { - return DatumFactory.createBool((Boolean) object.__tojava__(Boolean.class)); - } else if (object instanceof PyInteger) { - return DatumFactory.createInt4((Integer) object.__tojava__(Integer.class)); - } else if (object instanceof PyFloat) { - // J(P)ython is loosely typed, supports only float type, - // hence we convert everything to double to save precision - return DatumFactory.createFloat8((Double) object.__tojava__(Double.class)); - } else if (object instanceof PyString) { - return DatumFactory.createText((String) object.__tojava__(String.class)); - } else if (object instanceof PyNone) { - return DatumFactory.createNullDatum(); - } else if (object instanceof PyTuple) { - throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); - } else if (object instanceof PyList) { - throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); - } else if (object instanceof PyDictionary) { - throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); - } else { - Object javaObj = object.__tojava__(byte[].class); - if(javaObj instanceof byte[]) { - return DatumFactory.createBlob((byte[]) javaObj); - } - else { - throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); - } - } - } - - /** - * Convert a pyObject to a datum of the given type. - * @param object an object will be converted to a datum. - * @param type target datum type. - * @return a datum of the given type. - */ - public static Datum pyObjectToDatum(PyObject object, TajoDataTypes.Type type) { - return DatumFactory.cast(pyObjectToDatum(object), CatalogUtil.newSimpleDataType(type), null); - } +// /** +// * Convert a PyObject to a datum. +// * @param object +// * @return +// */ +// public static Datum pyObjectToDatum(PyObject object) { +// if (object instanceof PyLong) { +// return DatumFactory.createInt8((Long) object.__tojava__(Long.class)); +// } else if (object instanceof PyBoolean) { +// return DatumFactory.createBool((Boolean) object.__tojava__(Boolean.class)); +// } else if (object instanceof PyInteger) { +// return DatumFactory.createInt4((Integer) object.__tojava__(Integer.class)); +// } else if (object instanceof PyFloat) { +// // J(P)ython is loosely typed, supports only float type, +// // hence we convert everything to double to save precision +// return DatumFactory.createFloat8((Double) object.__tojava__(Double.class)); +// } else if (object instanceof PyString) { +// return DatumFactory.createText((String) object.__tojava__(String.class)); +// } else if (object instanceof PyNone) { +// return DatumFactory.createNullDatum(); +// } else if (object instanceof PyTuple) { +// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); +// } else if (object instanceof PyList) { +// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); +// } else if (object instanceof PyDictionary) { +// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); +// } else { +// Object javaObj = object.__tojava__(byte[].class); +// if(javaObj instanceof byte[]) { +// return DatumFactory.createBlob((byte[]) javaObj); +// } +// else { +// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); +// } +// } +// } +// +// /** +// * Convert a pyObject to a datum of the given type. +// * @param object an object will be converted to a datum. +// * @param type target datum type. +// * @return a datum of the given type. +// */ +// public static Datum pyObjectToDatum(PyObject object, TajoDataTypes.Type type) { +// return DatumFactory.cast(pyObjectToDatum(object), CatalogUtil.newSimpleDataType(type), null); +// } /** * Convert the primitive type to the Tajo type. diff --git a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java index fccaf2a109..9e7f334d91 100644 --- a/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java +++ b/tajo-storage/tajo-storage-common/src/test/java/org/apache/tajo/storage/TestLazyTuple.java @@ -65,7 +65,7 @@ public void setUp() { sb.append(DatumFactory.createFloat4(77.9f)).append('|'); sb.append(DatumFactory.createFloat8(271.9f)).append('|'); sb.append(DatumFactory.createText("str2")).append('|'); - sb.append(DatumFactory.createBlob("jinho".getBytes())).append('|'); + sb.append(DatumFactory.createBlob("jinho")).append('|'); sb.append(DatumFactory.createInet4("192.168.0.1")).append('|'); sb.append(new String(nullbytes)).append('|'); sb.append(NullDatum.get()); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java index 2db916afaf..e2d5481a55 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java @@ -68,7 +68,7 @@ public class TestDelimitedTextFile { DatumFactory.createFloat4(77.9f), // 5 DatumFactory.createFloat8(271.9d), // 6 DatumFactory.createText("hyunsik"), // 7 - DatumFactory.createBlob("hyunsik".getBytes()),// 8 + DatumFactory.createBlob("hyunsik"),// 8 DatumFactory.createInet4("192.168.0.1"), // 9 }); } diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java index 70282d9a9b..243807734d 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java @@ -91,7 +91,7 @@ public void testVarioutType() throws IOException { DatumFactory.createFloat4(77.9f), // 5 DatumFactory.createFloat8(271.9d), // 6 DatumFactory.createText("hyunsik"), // 7 - DatumFactory.createBlob("hyunsik".getBytes()), // 8 + DatumFactory.createBlob("hyunsik"), // 8 DatumFactory.createInet4("192.168.0.1"), // 9 NullDatum.get(), // 10 }); From 37d241385814ed08d0bea6ba2c062b73424977fd Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 7 Apr 2015 10:51:03 +0900 Subject: [PATCH 29/55] TAJO-1344_3 --- .../apache/tajo/plan/expr/FunctionInvoke.java | 5 +- .../tajo/plan/expr/GeneralFunctionEval.java | 6 +- .../plan/expr/LegacyScalarFunctionInvoke.java | 5 ++ .../tajo/plan/expr/PythonFunctionInvoke2.java | 82 +++---------------- 4 files changed, 22 insertions(+), 76 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java index 24e77a514b..3dd8b9f954 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java @@ -27,9 +27,10 @@ import org.apache.tajo.exception.UnsupportedException; import org.apache.tajo.storage.Tuple; +import java.io.Closeable; import java.io.IOException; -public abstract class FunctionInvoke { +public abstract class FunctionInvoke implements Closeable { @Expose protected FunctionDesc functionDesc; public FunctionInvoke(FunctionDesc functionDesc) { @@ -63,6 +64,8 @@ public static FunctionInvoke newInstance(FunctionDesc desc) throws InternalExcep */ public abstract Datum eval(Tuple tuple); + public abstract void close(); + @Override public boolean equals(Object o) { if (o instanceof FunctionInvoke) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 2758e5046f..a940ea27b8 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -40,8 +40,6 @@ public GeneralFunctionEval(@Nullable OverridableConf queryContext, FunctionDesc throws IOException { super(EvalType.FUNCTION, desc, givenArgs); this.queryContext = queryContext; -// this.funcInvoke = FunctionInvoke.newInstance(desc); -// this.funcInvoke.init(queryContext, getParamType()); } @Override @@ -60,7 +58,9 @@ public EvalNode bind(Schema schema) { @SuppressWarnings("unchecked") public Datum eval(Tuple tuple) { super.eval(tuple); - return funcInvoke.eval(evalParams(tuple)); + Datum res = funcInvoke.eval(evalParams(tuple)); + funcInvoke.close(); + return res; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java index ca1244d0bc..196ac0005a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java @@ -45,6 +45,11 @@ public Datum eval(Tuple tuple) { return function.eval(tuple); } + @Override + public void close() { + + } + @Override public boolean equals(Object o) { if (o instanceof LegacyScalarFunctionInvoke) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java index 70b68a8dbd..c0dc9bf3a9 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java @@ -89,13 +89,10 @@ public class PythonFunctionInvoke2 extends FunctionInvoke { private Schema inSchema; private Schema outSchema; private int [] projectionCols; - private boolean isBinded = false; private CSVLineSerDe lineSerDe = new CSVLineSerDe(); private TableMeta pipeMeta; - public static final String TURN_ON_OUTPUT_CAPTURING = "TURN_ON_OUTPUT_CAPTURING"; - public PythonFunctionInvoke2(FunctionDesc functionDesc) { super(functionDesc); if (!functionDesc.getInvocation().hasPython()) { @@ -134,23 +131,13 @@ private StreamingCommand startUdfController() throws IOException { process = processBuilder.start(); Runtime.getRuntime().addShutdownHook(new Thread(new ProcessKiller())); + LOG.info("process started"); return sc; } private String[] buildCommand() throws IOException { String[] command = new String[10]; -// String jarPath = conf.get("mapreduce.job.jar"); -// if (jarPath == null) { -// jarPath = conf.get(MRConfiguration.JAR); -// } -// String jobDir; -// if (jarPath != null) { -// jobDir = new File(jarPath).getParent(); -// } else { -// jobDir = ""; -// } - // String standardOutputRootWriteLocation = soc.getStandardOutputRootWriteLocation(); // String standardOutputRootWriteLocation = System.getProperty("tajo.log.dir"); // TODO @@ -158,18 +145,12 @@ private String[] buildCommand() throws IOException { String standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; -// if (execType.isLocal()) { -// controllerLogFileName = standardOutputRootWriteLocation + funcName + "_python.log"; -// outFileName = standardOutputRootWriteLocation + "cpython_" + funcName + "_" + ScriptingOutputCapturer.getRunId() + ".out"; -// errOutFileName = standardOutputRootWriteLocation + "cpython_" + funcName + "_" + ScriptingOutputCapturer.getRunId() + ".err"; -// } else { String funcName = invocationDesc.getName(); String filePath = invocationDesc.getPath(); - controllerLogFileName = standardOutputRootWriteLocation + funcName + "_python.log"; - outFileName = standardOutputRootWriteLocation + funcName + ".out"; - errOutFileName = standardOutputRootWriteLocation + funcName + ".err"; -// } + controllerLogFileName = standardOutputRootWriteLocation + funcName + "_python.log"; + outFileName = standardOutputRootWriteLocation + funcName + ".out"; + errOutFileName = standardOutputRootWriteLocation + funcName + ".err"; soc.registerOutputLocation(funcName, outFileName); @@ -321,29 +302,22 @@ public Datum eval(Tuple tuple) { return getOutput(tuple); } + @Override + public void close() { + process.destroy(); + LOG.info("process destroyed"); + } + private Datum getOutput(Tuple input) { if (outputQueue == null) { throw new RuntimeException("Process has already been shut down. No way to retrieve output for input: " + input); } -// if (ScriptingOutputCapturer.isClassCapturingOutput() && -// !soc.isInstanceCapturingOutput()) { -// Tuple t = TupleFactory.getInstance().newTuple(TURN_ON_OUTPUT_CAPTURING); -// try { -// inputQueue.put(t); -// } catch (InterruptedException e) { -// throw new RuntimeException("Failed adding capture input flag to inputQueue"); -// } -// soc.setInstanceCapturingOutput(true); -// } - try { -// if (this.inSchema == null || this.inSchema.size() == 0) { if (input == null) { //When nothing is passed into the UDF the tuple //being sent is the full tuple for the relation. //We want it to be nothing (since that's what the user wrote). -// input = TupleFactory.getInstance().newTuple(0); input = new VTuple(0); } LOG.info("input: " + input); @@ -371,8 +345,6 @@ private Datum getOutput(Tuple input) { throw new RuntimeException(outerrThreadsError); } -// Datum out = JythonUtils.objectToDatum(outSchema.getColumn(0).getDataType(), o); -// LOG.info("out: " + out); return (Datum) o; } @@ -501,38 +473,4 @@ public void run() { process.destroy(); } } -// -// public static void main(String[] args) throws IOException { -// String line; -// Scanner scan = new Scanner(System.in); -// -// Process process = Runtime.getRuntime ().exec("/bin/bash"); -// OutputStream stdin = process.getOutputStream(); -// InputStream stderr = process.getErrorStream(); -// InputStream stdout = process.getInputStream(); -// -// BufferedReader reader = new BufferedReader (new InputStreamReader(stdout)); -// BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(stdin)); -// -// while (scan.hasNext()) { -// String input = scan.nextLine(); -// if (input.trim().equals("exit")) { -// // Putting 'exit' amongst the echo --EOF--s below doesn't work. -// writer.write("exit\n"); -// } else { -//// writer.write("((" + input + ") && echo --EOF--) || echo --EOF--\n"); -// writer.write("((" + input + ") && echo --EOF--)\n"); -// } -// writer.flush(); -// -// line = reader.readLine(); -// while (line != null && ! line.trim().equals("--EOF--")) { -// System.out.println ("Stdout: " + line); -// line = reader.readLine(); -// } -// if (line == null) { -// break; -// } -// } -// } } From d464e9c4c39e80433109280ce84d6c5de7d7bb23 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 7 Apr 2015 18:49:31 +0900 Subject: [PATCH 30/55] TAJO-1344_3 --- .../apache/tajo/function/FunctionUtil.java | 10 + .../main/java/org/apache/tajo/QueryVars.java | 4 +- .../java/org/apache/tajo/datum/BlobDatum.java | 1 - .../org/apache/tajo/datum/DatumFactory.java | 12 +- .../org/apache/tajo/util/KeyValueSet.java | 2 +- .../tajo/engine/function/FunctionLoader.java | 11 +- .../tajo/engine/query/QueryContext.java | 8 + .../org/apache/tajo/master/GlobalEngine.java | 2 + .../apache/tajo/util/QueryContextUtil.java | 16 +- .../java/org/apache/tajo/worker/Task.java | 2 + .../src/main/resources/python/controller.py | 186 ++++------- .../org/apache/tajo/TajoTestingCluster.java | 3 +- .../apache/tajo/engine/eval/ExprTestBase.java | 2 + .../physical/TestExternalSortExec.java | 6 +- .../tajo/engine/query/TestSelectQuery.java | 6 + .../org/apache/tajo/plan/ExprAnnotator.java | 1 - .../tajo/plan/expr/GeneralFunctionEval.java | 14 +- .../plan/expr/GeneralPythonFunctionEval.java | 78 ----- .../tajo/plan/expr/PythonFunctionInvoke.java | 98 ------ .../{expr => function}/FunctionInvoke.java | 16 +- .../plan/function/FunctionInvokeContext.java | 43 +++ .../LegacyScalarFunctionInvoke.java | 8 +- .../plan/function/PythonFunctionInvoke.java | 51 +++ .../function/python/JythonScriptEngine.java | 315 ------------------ .../plan/function/python/JythonUtils.java | 253 -------------- .../function/python/PythonScriptEngine.java | 22 +- .../python/PythonScriptExecutor.java} | 192 ++++------- .../python/ScriptingOutputCapturer.java | 97 ------ .../function/python/TajoScriptEngine.java | 49 ++- .../function/stream/ByteBufLineReader.java | 22 +- .../plan/function/stream/InputHandler.java | 6 - .../plan/function/stream/OutputHandler.java | 18 +- .../function/stream/StreamingCommand.java | 230 ------------- .../stream/StreamingUDFInputHandler.java | 5 - .../stream/StreamingUDFOutputHandler.java | 5 - .../stream}/StreamingUtil.java | 11 +- .../TextFieldSerializerDeserializer.java | 4 +- 37 files changed, 340 insertions(+), 1469 deletions(-) rename tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java => tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java (65%) delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java rename tajo-plan/src/main/java/org/apache/tajo/plan/{expr => function}/FunctionInvoke.java (84%) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java rename tajo-plan/src/main/java/org/apache/tajo/plan/{expr => function}/LegacyScalarFunctionInvoke.java (88%) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java rename tajo-plan/src/main/java/org/apache/tajo/plan/{expr/PythonFunctionInvoke2.java => function/python/PythonScriptExecutor.java} (69%) delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java rename tajo-plan/src/main/java/org/apache/tajo/plan/{expr => function/stream}/StreamingUtil.java (89%) diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionUtil.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionUtil.java index dee5d1c138..ef70428af8 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionUtil.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/FunctionUtil.java @@ -18,6 +18,8 @@ package org.apache.tajo.function; +import org.apache.tajo.catalog.FunctionDesc; + import java.util.Collection; import static org.apache.tajo.common.TajoDataTypes.DataType; @@ -53,4 +55,12 @@ public static String buildParamTypeString(DataType [] paramTypes) { public static boolean isNullableParam(Class clazz) { return !clazz.isPrimitive(); } + + public static boolean isLegacyFunction(FunctionDesc desc) { + return desc.getInvocation().hasLegacy(); + } + + public static boolean isScriptFunction(FunctionDesc desc) { + return desc.getInvocation().hasPython(); + } } diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java index ba76d63f1f..c3835f8bfb 100644 --- a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java +++ b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -29,6 +29,8 @@ public enum QueryVars implements ConfigKey { OUTPUT_OVERWRITE, OUTPUT_AS_DIRECTORY, OUTPUT_PER_FILE_SIZE, + PYTHON_SCRIPT_CODE_DIR, + PYTHON_CONTROLLER_LOG_DIR ; QueryVars() { diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java index ef1323e333..cf190e2077 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/BlobDatum.java @@ -131,7 +131,6 @@ public boolean equals(Object obj) { BlobDatum other = (BlobDatum) obj; initFromBytes(); other.initFromBytes(); -// return bb.equals(other.bb); return Arrays.equals(this.val, other.val); } diff --git a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java index 0032b5c95d..bd1b88fa6b 100644 --- a/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java +++ b/tajo-common/src/main/java/org/apache/tajo/datum/DatumFactory.java @@ -378,16 +378,16 @@ public static TimestampDatum parseTimestamp(String str, @Nullable TimeZone tz) { return new TimestampDatum(DateTimeUtil.toJulianTimestampWithTZ(str, tz)); } - public static BlobDatum createBlob(byte[] val) { - return new BlobDatum(val); + public static BlobDatum createBlob(byte[] encoded) { + return new BlobDatum(encoded); } - public static BlobDatum createBlob(byte[] val, int offset, int length) { - return new BlobDatum(val, offset, length); + public static BlobDatum createBlob(byte[] encoded, int offset, int length) { + return new BlobDatum(encoded, offset, length); } - public static BlobDatum createBlob(String val) { - return new BlobDatum(Base64.encodeBase64(val.getBytes())); + public static BlobDatum createBlob(String plainString) { + return new BlobDatum(Base64.encodeBase64(plainString.getBytes())); } public static Inet4Datum createInet4(int encoded) { diff --git a/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java b/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java index 6af0c9e9da..5dba9e2c9b 100644 --- a/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java +++ b/tajo-common/src/main/java/org/apache/tajo/util/KeyValueSet.java @@ -104,7 +104,7 @@ public String get(String key, String defaultVal) { } else if (defaultVal != null) { return defaultVal; } else { - throw new IllegalArgumentException("No such a config key: " + key); + throw new IllegalArgumentException("No such config key: " + key); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index 18a61da9a1..506fc21c48 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.FunctionDesc; @@ -88,17 +89,21 @@ public static Map loadOptionalFunctions(TajoCon Path codePath = new Path(codePathStr); List filePaths = TUtil.newList(); if (localFS.isDirectory(codePath)) { - for (FileStatus file : localFS.listStatus(codePath)) { + for (FileStatus file : localFS.listStatus(codePath, new PathFilter() { + @Override + public boolean accept(Path path) { + return path.getName().endsWith(PythonScriptEngine.FILE_EXTENSION); + } + })) { filePaths.add(file.getPath()); } } else { filePaths.add(codePath); } for (Path filePath : filePaths) { - for (FunctionDesc f : PythonScriptEngine.registerFunctions(filePath.toString(), + for (FunctionDesc f : PythonScriptEngine.registerFunctions(filePath.toUri(), FunctionLoader.PYTHON_FUNCTION_NAMESPACE)) { functionMap.put(f.getSignature(), f); -// LOG.info(f); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java index 7b3c00db30..0e595f79f3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java @@ -151,6 +151,14 @@ public NodeType getCommandType() { return strVal != null ? NodeType.valueOf(strVal) : null; } + public void setPythonScriptPath(String path) { + put(QueryVars.PYTHON_SCRIPT_CODE_DIR, path); + } + + public String getPythonScriptPath() { + return get(QueryVars.PYTHON_SCRIPT_CODE_DIR); + } + public void setCreateTable() { setCommandType(NodeType.CREATE_TABLE); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java index 4e87d35c38..6bbab22f5e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java @@ -50,6 +50,7 @@ import org.apache.tajo.plan.verifier.VerifyException; import org.apache.tajo.storage.StorageManager; import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.QueryContextUtil; import java.io.IOException; import java.sql.SQLException; @@ -147,6 +148,7 @@ private QueryContext createQueryContext(Session session) { public SubmitQueryResponse executeQuery(Session session, String query, boolean isJson) { LOG.info("Query: " + query); QueryContext queryContext = createQueryContext(session); + QueryContextUtil.updatePythonScriptPath(context.getConf(), queryContext); Expr planningContext; try { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java b/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java similarity index 65% rename from tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java rename to tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java index 0401d98d74..dbee78e5ba 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonConstants.java +++ b/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,11 +16,13 @@ * limitations under the License. */ -package org.apache.tajo.plan.function.python; +package org.apache.tajo.util; -public class JythonConstants { - public final static String SKIP_TOKEN = "__"; - public final static String OUTPUT_TYPE = "outputType".intern(); - public final static String OUTPUT_SCHEMA_FUNCTION = "outputSchemaFunction".intern(); - public final static String SCHEMA_FUNCTION = "schemaFunction".intern(); +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; + +public class QueryContextUtil { + public static void updatePythonScriptPath(TajoConf systemConf, QueryContext queryContext) { + queryContext.setPythonScriptPath(systemConf.getVar(TajoConf.ConfVars.PYTHON_CODE_DIR)); + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index b08af2b3a7..bec22b962e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -58,6 +58,7 @@ import org.apache.tajo.util.NetUtils; import io.netty.handler.codec.http.QueryStringDecoder; +import org.apache.tajo.util.QueryContextUtil; import java.io.File; import java.io.IOException; @@ -135,6 +136,7 @@ public Task(String taskRunnerId, } public void initPlan() throws IOException { + QueryContextUtil.updatePythonScriptPath(systemConf, queryContext); plan = LogicalNodeDeserializer.deserialize(queryContext, request.getPlan()); LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); if (scanNode != null) { diff --git a/tajo-core/src/main/resources/python/controller.py b/tajo-core/src/main/resources/python/controller.py index 4b7f31b345..e64aa3ee88 100644 --- a/tajo-core/src/main/resources/python/controller.py +++ b/tajo-core/src/main/resources/python/controller.py @@ -18,6 +18,7 @@ import sys import os import logging +import base64 from datetime import datetime try: @@ -26,7 +27,7 @@ except ImportError: USE_DATEUTIL = False -from tajo_util import write_user_exception, udf_logging, outputType +from tajo_util import write_user_exception, udf_logging FIELD_DELIMITER = ',' TUPLE_START = '(' @@ -76,91 +77,82 @@ class PythonStreamingController: def __init__(self, profiling_mode=False): self.profiling_mode = profiling_mode - self.input_count = 0 - self.next_input_count_to_log = 1 - def main(self, module_name, file_path, func_name, cache_path, - output_stream_path, error_stream_path, log_file_name, is_illustrate_str): + output_stream_path, error_stream_path, log_file_name, output_schema): sys.stdin = os.fdopen(sys.stdin.fileno(), 'rb', 0) - #Need to ensure that user functions can't write to the streams we use to - #communicate with pig. + # Need to ensure that user functions can't write to the streams we use to communicate with pig. self.stream_output = os.fdopen(sys.stdout.fileno(), 'wb', 0) self.stream_error = os.fdopen(sys.stderr.fileno(), 'wb', 0) self.input_stream = sys.stdin - self.log_stream = open(output_stream_path, 'a') - sys.stderr = open(error_stream_path, 'w') - is_illustrate = is_illustrate_str == "true" + # TODO: support controller logging + # self.log_stream = open(output_stream_path, 'a') + # sys.stderr = open(error_stream_path, 'w') sys.path.append(file_path) sys.path.append(cache_path) sys.path.append('.') logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) - logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") + logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the " + "INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") input_str = self.get_next_input() - logging.info('main: ' + input_str) + logging.info('1: ' + input_str) try: - # logging.info('module: ' + module_name + ' func_name: ' + func_name) - # logging.info(globals()) - # logging.info(locals()) func = __import__(module_name, globals(), locals(), [func_name], -1).__dict__[func_name] - logging.info("imported") except: - #These errors should always be caused by user code. + # These errors should always be caused by user code. write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-1) - if is_illustrate or udf_logging.udf_log_level != logging.DEBUG: - #Only log output for illustrate after we get the flag to capture output. - sys.stdout = open(os.devnull, 'w') - else: - sys.stdout = self.log_stream + logging.info('2: ') + # if udf_logging.udf_log_level != logging.DEBUG: + # #Only log output for illustrate after we get the flag to capture output. + # sys.stdout = open(os.devnull, 'w') + # else: + # sys.stdout = self.log_stream - while input_str != END_OF_STREAM: - logging.info('while loop') - should_log = False - if self.input_count == self.next_input_count_to_log: - should_log = True - log_message = logging.info - self.update_next_input_count_to_log() - elif udf_logging.udf_log_level == logging.DEBUG: - should_log = True - log_message = logging.debug + should_log = True + log_message = logging.info + if udf_logging.udf_log_level == logging.DEBUG: + should_log = True + log_message = logging.debug + logging.info('3: ') + while input_str != END_OF_STREAM: try: try: if should_log: - log_message("Row %s: Serialized Input: %s" % (self.input_count, input_str)) + log_message("Serialized Input: %s" % (input_str)) inputs = deserialize_input(input_str) if should_log: - log_message("Row %s: Deserialized Input: %s" % (self.input_count, unicode(inputs))) + log_message("Deserialized Input: %s" % (unicode(inputs))) except: - #Capture errors where the user passes in bad data. + # Capture errors where the user passes in bad data. write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-3) try: func_output = func(*inputs) if should_log: - log_message("Row %s: UDF Output: %s" % (self.input_count, unicode(func_output))) + log_message("UDF Output: %s" % (unicode(func_output))) except: - #These errors should always be caused by user code. + # These errors should always be caused by user code. write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-2) - output = serialize_output(func_output) + output = serialize_output(func_output, output_schema) if should_log: - log_message("Row %s: Serialized Output: %s" % (self.input_count, output)) + log_message("Serialized Output: %s" % (output)) self.stream_output.write( "%s%s" % (output, END_RECORD_DELIM) ) except Exception as e: - #This should only catch internal exceptions with the controller - #and pig- not with user code. + # This should only catch internal exceptions with the controller + # and pig- not with user code. import traceback traceback.print_exc(file=self.stream_error) sys.exit(-3) @@ -174,47 +166,34 @@ def main(self, def get_next_input(self): input_stream = self.input_stream - log_stream = self.log_stream + # log_stream = self.log_stream - logging.info('test') input_str = input_stream.readline() - logging.info('input_str: ' + input_str) + logging.info('get_next_input1: ' + input_str) while input_str.endswith(END_RECORD_DELIM) == False: line = input_stream.readline() - logging.info('line: ' + line) if line == '': input_str = '' break input_str += line + logging.info('get_next_input2: ' + input_str) + if input_str == '': return END_OF_STREAM - if input_str == TURN_ON_OUTPUT_CAPTURING: - logging.debug("Turned on Output Capturing") - sys.stdout = log_stream - return self.get_next_input() + # if input_str == TURN_ON_OUTPUT_CAPTURING: + # logging.debug("Turned on Output Capturing") + # sys.stdout = log_stream + # return self.get_next_input() if input_str == END_OF_STREAM: return input_str - self.input_count += 1 - + logging.info('get_next_input3: ' + input_str) return input_str[:-END_RECORD_DELIM_LENGTH] - def update_next_input_count_to_log(self): - """ - Want to log enough rows that you can see progress being made and see timings without wasting time logging thousands of rows. - Show first 10 rows, and then the first 5 rows of every order of magnitude (10-15, 100-105, 1000-1005, ...) - """ - if self.next_input_count_to_log < 10: - self.next_input_count_to_log = self.next_input_count_to_log + 1 - elif self.next_input_count_to_log % 10 == 5: - self.next_input_count_to_log = (self.next_input_count_to_log - 5) * 10 - else: - self.next_input_count_to_log = self.next_input_count_to_log + 1 - def close_controller(self, exit_code): sys.stderr.close() self.stream_error.write("\n") @@ -228,19 +207,12 @@ def deserialize_input(input_str): if len(input_str) == 0: return [] - logging.info('deserialize_input: ' + input_str) - # [logging.info(param) for param in input_str.split(WRAPPED_PARAMETER_DELIMITER)] - # return [_deserialize_input(param, 0, len(param)) for param in input_str.split(WRAPPED_PARAMETER_DELIMITER)] - [logging.info(param) for param in input_str.split(WRAPPED_FIELD_DELIMITER)] return [_deserialize_input(param, 0, len(param)) for param in input_str.split(WRAPPED_FIELD_DELIMITER)] def _deserialize_input(input_str, si, ei): - logging.info('_deserialize_input: ' + input_str) - logging.info(si) - logging.info(ei) len = ei - si + 1 if len < 1: - #Handle all of the cases where you can have valid empty input. + # Handle all of the cases where you can have valid empty input. if ei == si: if input_str[si] == TYPE_CHARARRAY: return u"" @@ -255,39 +227,6 @@ def _deserialize_input(input_str, si, ei): schema = tokens[0]; param = tokens[1]; - # first = input_str[si] - # schema = input_str[si+1] if first == PRE_WRAP_DELIM else first - # logging.info('first: ' + first) - logging.info('schema: ' + schema) - logging.info('param: ' + param) - - # if schema == NULL_BYTE: - # return None - # elif schema == TYPE_TUPLE or schema == TYPE_MAP or schema == TYPE_BAG: - # return _deserialize_collection(input_str, schema, si+3, ei-3) - # elif schema == TYPE_CHARARRAY: - # return unicode(input_str[si+1:ei+1], 'utf-8') - # elif schema == TYPE_BYTEARRAY: - # return bytearray(input_str[si+1:ei+1]) - # elif schema == TYPE_INTEGER: - # return int(input_str[si+1:ei+1]) - # elif schema == TYPE_LONG or schema == TYPE_BIGINTEGER: - # return long(input_str[si+1:ei+1]) - # elif schema == TYPE_FLOAT or schema == TYPE_DOUBLE or schema == TYPE_BIGDECIMAL: - # return float(input_str[si+1:ei+1]) - # elif schema == TYPE_BOOLEAN: - # return input_str[si+1:ei+1] == "true" - # elif schema == TYPE_DATETIME: - # #Format is "yyyy-MM-ddTHH:mm:ss.SSS+00:00" or "2013-08-23T18:14:03.123+ZZ" - # if USE_DATEUTIL: - # return parser.parse(input_str[si+1:ei+1]) - # else: - # #Try to use datetime even though it doesn't handle time zones properly, - # #We only use the first 3 microsecond digits and drop time zone (first 23 characters) - # return datetime.strptime(input_str[si+1:si+24], "%Y-%m-%dT%H:%M:%S.%f") - # else: - # raise Exception("Can't determine type of input: %s" % input_str[si:ei+1]) - if schema == NULL_BYTE: return None elif schema == TYPE_CHARARRAY: @@ -303,12 +242,12 @@ def _deserialize_input(input_str, si, ei): elif schema == TYPE_BOOLEAN: return param == "true" elif schema == TYPE_DATETIME: - #Format is "yyyy-MM-ddTHH:mm:ss.SSS+00:00" or "2013-08-23T18:14:03.123+ZZ" + # Format is "yyyy-MM-ddTHH:mm:ss.SSS+00:00" or "2013-08-23T18:14:03.123+ZZ" if USE_DATEUTIL: return parser.parse(param) else: - #Try to use datetime even though it doesn't handle time zones properly, - #We only use the first 3 microsecond digits and drop time zone (first 23 characters) + # Try to use datetime even though it doesn't handle time zones properly, + # We only use the first 3 microsecond digits and drop time zone (first 23 characters) return datetime.strptime(param, "%Y-%m-%dT%H:%M:%S.%f") else: raise Exception("Can't determine type of input: %s" % param) @@ -375,7 +314,7 @@ def wrap_tuple(o, serialized_item): else: return serialized_item -def serialize_output(output, utfEncodeAllFields=False): +def serialize_output(output, out_schema, utfEncodeAllFields=False): """ @param utfEncodeStrings - Generally we want to utf encode only strings. But for Maps we utf encode everything because on the Java side we don't know the schema @@ -385,30 +324,23 @@ def serialize_output(output, utfEncodeAllFields=False): output_type = type(output) if output is None: - return WRAPPED_NULL_BYTE - elif output_type == tuple: - return (WRAPPED_TUPLE_START + - WRAPPED_FIELD_DELIMITER.join([serialize_output(o, utfEncodeAllFields) for o in output]) + - WRAPPED_TUPLE_END) - elif output_type == list: - return (WRAPPED_BAG_START + - WRAPPED_FIELD_DELIMITER.join([wrap_tuple(o, serialize_output(o, utfEncodeAllFields)) for o in output]) + - WRAPPED_BAG_END) - elif output_type == dict: - return (WRAPPED_MAP_START + - WRAPPED_FIELD_DELIMITER.join(['%s%s%s' % (k.encode('utf-8'), MAP_KEY, serialize_output(v, True)) for k, v in output.iteritems()]) + - WRAPPED_MAP_END) + result = WRAPPED_NULL_BYTE elif output_type == bool: - return ("true" if output else "false") + result = ("true" if output else "false") elif output_type == bytearray: - return str(output) + result = str(output) elif output_type == datetime: - return output.isoformat() + result = output.isoformat() elif utfEncodeAllFields or output_type == str or output_type == unicode: - #unicode is necessary in cases where we're encoding non-strings. - return unicode(output).encode('utf-8') + # unicode is necessary in cases where we're encoding non-strings. + result = unicode(output).encode('utf-8') + else: + result = str(output) + + if out_schema == "blob": + return base64.b64encode(result) else: - return str(output) + return result if __name__ == '__main__': controller = PythonStreamingController() diff --git a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java index 8dfab33e1f..39ff891ca6 100644 --- a/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java +++ b/tajo-core/src/test/java/org/apache/tajo/TajoTestingCluster.java @@ -57,6 +57,7 @@ import java.io.IOException; import java.io.Writer; import java.net.InetSocketAddress; +import java.net.URISyntaxException; import java.sql.ResultSet; import java.util.ArrayList; import java.util.List; @@ -160,7 +161,7 @@ void initPropertiesAndConfigs() { // Memory cache termination conf.setIntVar(ConfVars.WORKER_HISTORY_EXPIRE_PERIOD, 1); - conf.setStrings(ConfVars.PYTHON_CODE_DIR.varname, "python/test_funcs.py", "python/test_funcs2.py"); + conf.setStrings(ConfVars.PYTHON_CODE_DIR.varname, getClass().getResource("/python").toString()); /* Since Travi CI limits the size of standard output log up to 4MB */ if (!StringUtils.isEmpty(LOG_LEVEL)) { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 99b6f037ce..80f32b16fc 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -53,6 +53,7 @@ import org.apache.tajo.util.BytesUtils; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.KeyValueSet; +import org.apache.tajo.util.QueryContextUtil; import org.apache.tajo.util.datetime.DateTimeUtil; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -223,6 +224,7 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S queryContext = LocalTajoTestingUtility.createDummyContext(conf); queryContext.putAll(context); } + QueryContextUtil.updatePythonScriptPath(conf, queryContext); String timezoneId = queryContext.get(SessionVars.TIMEZONE); TimeZone timeZone = TimeZone.getTimeZone(timezoneId); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java index 946e0f30bf..0d006e05e5 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/planner/physical/TestExternalSortExec.java @@ -61,7 +61,7 @@ public class TestExternalSortExec { private LogicalPlanner planner; private Path testDir; - private final int numTuple = 3000000; + private final int numTuple = 100000; private Random rnd = new Random(System.currentTimeMillis()); private TableDesc employee; @@ -141,7 +141,11 @@ public final void testNext() throws IOException, PlanningException { SeqScanExec scan = sortExec.getChild(); ExternalSortExec extSort = new ExternalSortExec(ctx, ((MemSortExec)sortExec).getPlan(), scan); + extSort.setSortBufferBytesNum(1024*1024); proj.setChild(extSort); + } else { + ExternalSortExec extSort = proj.getChild(); + extSort.setSortBufferBytesNum(1024*1024); } Tuple tuple; diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 0a3f19926b..219bb7962e 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -716,6 +716,12 @@ public void testSelectPythonFuncs() throws Exception { cleanupQuery(res); } + @Test + public void testSelectPythonFuncs2() throws Exception { + executeString("select sum_py(n_nationkey, n_regionkey) as sum\n" + + "from nation where n_nationkey < 5"); + } + @Test public void testSelectWithPredicateOnPythonFunc() throws Exception { ResultSet res = executeQuery(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index d2a2138e07..e9638d2002 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -34,7 +34,6 @@ import org.apache.tajo.plan.algebra.BaseAlgebraVisitor; import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.function.AggFunction; -import org.apache.tajo.plan.function.GeneralFunction; import org.apache.tajo.plan.logical.NodeType; import org.apache.tajo.plan.nameresolver.NameResolver; import org.apache.tajo.plan.nameresolver.NameResolvingMode; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index a940ea27b8..7c17982fc4 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -24,22 +24,22 @@ import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; -import org.apache.tajo.exception.InternalException; -import org.apache.tajo.plan.function.GeneralFunction; +import org.apache.tajo.plan.function.FunctionInvoke; +import org.apache.tajo.plan.function.FunctionInvokeContext; import org.apache.tajo.storage.Tuple; import org.apache.tajo.util.TUtil; -import javax.annotation.Nullable; import java.io.IOException; +import java.util.Map; public class GeneralFunctionEval extends FunctionEval { @Expose protected FunctionInvoke funcInvoke; - @Expose protected OverridableConf queryContext; + @Expose protected FunctionInvokeContext invokeContext; - public GeneralFunctionEval(@Nullable OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) + public GeneralFunctionEval(OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) throws IOException { super(EvalType.FUNCTION, desc, givenArgs); - this.queryContext = queryContext; + this.invokeContext = new FunctionInvokeContext(queryContext, getParamType()); } @Override @@ -47,7 +47,7 @@ public EvalNode bind(Schema schema) { super.bind(schema); try { this.funcInvoke = FunctionInvoke.newInstance(funcDesc); - this.funcInvoke.init(queryContext, getParamType()); + this.funcInvoke.init(invokeContext); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java deleted file mode 100644 index 05453dc9e5..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralPythonFunctionEval.java +++ /dev/null @@ -1,78 +0,0 @@ -///** -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ -// -//package org.apache.tajo.plan.expr; -// -//import org.apache.tajo.catalog.FunctionDesc; -//import org.apache.tajo.catalog.Schema; -//import org.apache.tajo.common.TajoDataTypes; -//import org.apache.tajo.datum.Datum; -//import org.apache.tajo.function.PythonInvocationDesc; -//import org.apache.tajo.plan.function.python.JythonScriptEngine; -//import org.apache.tajo.plan.function.python.JythonUtils; -//import org.apache.tajo.storage.Tuple; -//import org.apache.tajo.storage.VTuple; -//import org.python.core.PyFunction; -//import org.python.core.PyObject; -// -//import java.io.IOException; -// -///** -// * Python implementation of a Tajo UDF performs mappings between Python & Tajo data structures. -// */ -//public class GeneralPythonFunctionEval extends FunctionEval { -// private Tuple params = null; -// -// public GeneralPythonFunctionEval(FunctionDesc funcDesc, EvalNode[] argEvals) { -// super(EvalType.PYTHON_FUNCTION, funcDesc, argEvals); -// } -// -// @Override -// public Datum eval(Schema schema, Tuple tuple) { -// if (this.params == null) { -// params = new VTuple(argEvals.length); -// } -// if(argEvals != null) { -// params.clear(); -// for(int i=0;i < argEvals.length; i++) { -// params.put(i, argEvals[i].eval(schema, tuple)); -// } -// } -// -// -// PythonInvocationDesc invokeDesc = funcDesc.getInvocation().getPython(); -// try { -// PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); -// -// TajoDataTypes.DataType[] paramTypes = funcDesc.getSignature().getParamTypes(); -// PyObject result; -// if (paramTypes.length == 0) { -// result = function.__call__(); -// } else { -// // Find the actual data types from the given parameters at runtime, -// // and convert them into PyObject instances. -// PyObject[] pyParams = JythonUtils.tupleToPyTuple(params).getArray(); -// result = function.__call__(pyParams); -// } -// -// return JythonUtils.pyObjectToDatum(result); -// } catch (IOException e) { -// throw new RuntimeException(e); -// } -// } -//} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java deleted file mode 100644 index 0313874140..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke.java +++ /dev/null @@ -1,98 +0,0 @@ -///** -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ -// -//package org.apache.tajo.plan.expr; -// -//import com.google.common.base.Objects; -//import com.google.gson.annotations.Expose; -//import org.apache.tajo.OverridableConf; -//import org.apache.tajo.catalog.FunctionDesc; -//import org.apache.tajo.catalog.Schema; -//import org.apache.tajo.common.TajoDataTypes; -//import org.apache.tajo.datum.Datum; -//import org.apache.tajo.function.PythonInvocationDesc; -//import org.apache.tajo.plan.function.python.JythonScriptEngine; -//import org.apache.tajo.plan.function.python.JythonUtils; -//import org.apache.tajo.storage.Tuple; -//import org.apache.tajo.util.TUtil; -//import org.python.core.PyFunction; -//import org.python.core.PyObject; -// -//import java.io.IOException; -//import java.util.Arrays; -// -//public class PythonFunctionInvoke extends FunctionInvoke { -// @Expose private PythonInvocationDesc invokeDesc; -// @Expose private TajoDataTypes.DataType[] paramTypes; -// -// public PythonFunctionInvoke(FunctionDesc funcDesc) { -// super(funcDesc); -// this.invokeDesc = funcDesc.getInvocation().getPython(); -// this.paramTypes = funcDesc.getSignature().getParamTypes(); -// } -// -// @Override -// public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { -// // nothing to do -// } -// -// @Override -// public Datum eval(Tuple tuple) { -// try { -// PyFunction function = JythonScriptEngine.getFunction(invokeDesc.getPath(), invokeDesc.getName()); -// -// PyObject result; -// if (paramTypes.length == 0) { -// result = function.__call__(); -// } else { -// // Find the actual data types from the given parameters at runtime, -// // and convert them into PyObject instances. -// PyObject[] pyParams = JythonUtils.tupleToPyTuple(tuple).getArray(); -// result = function.__call__(pyParams); -// } -// -// return JythonUtils.pyObjectToDatum(result); -// } catch (IOException e) { -// throw new RuntimeException(e); -// } -// } -// -// @Override -// public boolean equals(Object o) { -// if (o instanceof PythonFunctionInvoke) { -// PythonFunctionInvoke other = (PythonFunctionInvoke) o; -// return this.invokeDesc.equals(other.invokeDesc) && -// TUtil.checkEquals(this.paramTypes, other.paramTypes); -// } -// return false; -// } -// -// @Override -// public int hashCode() { -// return Objects.hashCode(invokeDesc, Arrays.hashCode(paramTypes)); -// } -// -// @Override -// public Object clone() throws CloneNotSupportedException { -// PythonFunctionInvoke clone = (PythonFunctionInvoke) super.clone(); -// clone.invokeDesc = (PythonInvocationDesc) this.invokeDesc.clone(); -// clone.paramTypes = new TajoDataTypes.DataType[paramTypes.length]; -// paramTypes = Arrays.copyOf(paramTypes, paramTypes.length); -// return clone; -// } -//} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java similarity index 84% rename from tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java index 3dd8b9f954..609c9102e7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java @@ -16,12 +16,10 @@ * limitations under the License. */ -package org.apache.tajo.plan.expr; +package org.apache.tajo.plan.function; import com.google.gson.annotations.Expose; -import org.apache.tajo.OverridableConf; import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; import org.apache.tajo.exception.InternalException; import org.apache.tajo.exception.UnsupportedException; @@ -41,21 +39,13 @@ public static FunctionInvoke newInstance(FunctionDesc desc) throws InternalExcep if (desc.getInvocation().hasLegacy()) { return new LegacyScalarFunctionInvoke(desc); } else if (desc.getInvocation().hasPython()) { -// return new PythonFunctionInvoke(desc); - return new PythonFunctionInvoke2(desc); + return new PythonFunctionInvoke(desc); } else { throw new UnsupportedException(desc.getInvocation() + " is not supported"); } } - public abstract void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) throws IOException; - - -// /** -// * Bind an input schema. -// * @param schema intpu schema -// */ -// public abstract void bind(Schema schema); + public abstract void init(FunctionInvokeContext context) throws IOException; /** * Evaluate the given tuple with a function diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java new file mode 100644 index 0000000000..0e2c3002b5 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function; + +import org.apache.tajo.OverridableConf; +import org.apache.tajo.plan.expr.FunctionEval; +import org.apache.tajo.util.KeyValueSet; + +import java.util.Map; + +public class FunctionInvokeContext { + private final OverridableConf queryContext; + private final FunctionEval.ParamType[] paramTypes; + + public FunctionInvokeContext(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { + this.queryContext = queryContext; + this.paramTypes = paramTypes; + } + + public OverridableConf getQueryContext() { + return queryContext; + } + + public FunctionEval.ParamType[] getParamTypes() { + return paramTypes; + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java similarity index 88% rename from tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java index 196ac0005a..a1d2ad6d05 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/LegacyScalarFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java @@ -16,14 +16,12 @@ * limitations under the License. */ -package org.apache.tajo.plan.expr; +package org.apache.tajo.plan.function; import com.google.gson.annotations.Expose; -import org.apache.tajo.OverridableConf; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.datum.Datum; import org.apache.tajo.exception.InternalException; -import org.apache.tajo.plan.function.GeneralFunction; import org.apache.tajo.storage.Tuple; import org.apache.tajo.util.TUtil; @@ -36,8 +34,8 @@ public LegacyScalarFunctionInvoke(FunctionDesc funcDesc) throws InternalExceptio } @Override - public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { - function.init(queryContext, paramTypes); + public void init(FunctionInvokeContext context) { + function.init(context.getQueryContext(), context.getParamTypes()); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java new file mode 100644 index 0000000000..5721dc3541 --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -0,0 +1,51 @@ +/* + * Lisensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.plan.function; + +import org.apache.tajo.catalog.FunctionDesc; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.plan.function.python.PythonScriptExecutor; +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; + +public class PythonFunctionInvoke extends FunctionInvoke { + + private PythonScriptExecutor scriptExecutor; + + public PythonFunctionInvoke(FunctionDesc functionDesc) { + super(functionDesc); + scriptExecutor = new PythonScriptExecutor(functionDesc); + } + + @Override + public void init(FunctionInvokeContext context) throws IOException { + scriptExecutor.start(context); + } + + @Override + public Datum eval(Tuple tuple) { + return scriptExecutor.eval(tuple); + } + + @Override + public void close() { + scriptExecutor.stop(); + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java deleted file mode 100644 index 0888cf61f6..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonScriptEngine.java +++ /dev/null @@ -1,315 +0,0 @@ -///* -// * Licensed to the Apache Software Foundation (ASF) under one -// * or more contributor license agreements. See the NOTICE file -// * distributed with this work for additional information -// * regarding copyright ownership. The ASF licenses this file -// * to you under the Apache License, Version 2.0 (the -// * "License"); you may not use this file except in compliance -// * with the License. You may obtain a copy of the License at -// * -// * http://www.apache.org/licenses/LICENSE-2.0 -// * -// * Unless required by applicable law or agreed to in writing, software -// * distributed under the License is distributed on an "AS IS" BASIS, -// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// * See the License for the specific language governing permissions and -// * limitations under the License. -// */ -// -//package org.apache.tajo.plan.function.python; -// -//import java.io.File; -//import java.io.IOException; -//import java.io.InputStream; -//import java.util.ArrayList; -//import java.util.HashMap; -//import java.util.List; -//import java.util.Map; -//import java.util.Set; -// -//import org.apache.commons.logging.Log; -//import org.apache.commons.logging.LogFactory; -//import org.apache.tajo.catalog.FunctionDesc; -//import org.apache.tajo.catalog.proto.CatalogProtos; -//import org.apache.tajo.common.TajoDataTypes; -//import org.apache.tajo.exception.UnsupportedException; -//import org.apache.tajo.function.FunctionInvocation; -//import org.apache.tajo.function.FunctionSignature; -//import org.apache.tajo.function.FunctionSupplement; -//import org.apache.tajo.function.PythonInvocationDesc; -//import org.apache.tajo.util.TUtil; -//import org.python.core.*; -//import org.python.util.PythonInterpreter; -// -///** -// * Implementation of the script engine for Jython -// */ -//public class JythonScriptEngine extends TajoScriptEngine { -// private static final Log LOG = LogFactory.getLog(JythonScriptEngine.class); -// public static final String NAMESPACE_SEPARATOR = "."; -// -// /** -// * Language Interpreter Uses static holder pattern -// */ -// private static class Interpreter { -// static final PythonInterpreter interpreter; -// static final ArrayList filesLoaded = new ArrayList(); -// static final String JVM_JAR; -// -// static { -// // should look like: file:JVM_JAR!/java/lang/Object.class -// String rpath = Object.class.getResource("Object.class").getPath(); -// JVM_JAR = rpath.replaceAll("^file:(.*)!/java/lang/Object.class$", "$1"); -// -// // Determine if a usable python.cachedir has been provided -// // if not, certain uses of jython's import will not work e.g., so create a tmp dir -// // - from some.package import * -// // - import non.jvm.package -// try { -// String skip = System.getProperty(PySystemState.PYTHON_CACHEDIR_SKIP, "false"); -// if (skip.equalsIgnoreCase("true")) { -// LOG.warn("jython cachedir skipped, jython may not work"); -// } else { -// File tmp = null; -// String cdir = System.getProperty(PySystemState.PYTHON_CACHEDIR); -// if (cdir != null) { -// tmp = new File(cdir); -// if (!tmp.canWrite()) { -// LOG.error("CACHEDIR: not writable"); -// throw new RuntimeException("python.cachedir not writable: " + cdir); -// } -// } -// if (tmp == null) { -// tmp = File.createTempFile("tajo_jython_", ""); -// tmp.delete(); -// if (!tmp.mkdirs()) { -// LOG.warn("unable to create a tmp dir for the cache, jython may not work"); -// } else { -// LOG.info("created tmp python.cachedir=" + tmp); -// System.setProperty(PySystemState.PYTHON_CACHEDIR, tmp.getAbsolutePath()); -// } -// Runtime.getRuntime().addShutdownHook(new DirDeleter(tmp)); -// } -// } -// // local file system import path elements: current dir, JYTHON_HOME/Lib -// Py.getSystemState().path.append(new PyString(System.getProperty("user.dir"))); -// String jyhome = System.getenv("JYTHON_HOME"); -// if (jyhome != null) { -// Py.getSystemState().path.append(new PyString(jyhome + File.separator + "Lib")); -// } -// } catch (Exception e) { -// LOG.warn("issue with jython cache dir", e); -// } -// -// // cacdedir now configured, allocate the python interpreter -// interpreter = new PythonInterpreter(); -// } -// -// /** -// * Ensure the decorator functions are defined in the interpreter, and -// * manage the module import dependencies. -// * @param initPhase True if the script is not registered. Otherwise false. -// * @param path location of a script file to exec in the interpreter -// * @throws IOException -// */ -// static synchronized void init(boolean initPhase, String path) throws IOException { -// // Decorators - -// // "outputType" -// -// if (!filesLoaded.contains(path)) { -// // attempt addition of type decorator handler, fail silently -// interpreter.exec("def outputType(type_def):\n" -// + " def decorator(func):\n" -// + " func.outputType = type_def\n" -// + " return func\n" -// + " return decorator\n\n"); -// -// // TODO: Currently, we don't support the customized output type feature. -//// interpreter.exec("def outputSchemaFunction(schema_def):\n" -//// + " def decorator(func):\n" -//// + " func.outputSchemaFunction = schema_def\n" -//// + " return func\n" -//// + " return decorator\n"); -//// -//// interpreter.exec("def schemaFunction(schema_def):\n" -//// + " def decorator(func):\n" -//// + " func.schemaFunction = schema_def\n" -//// + " return func\n" -//// + " return decorator\n\n"); -// -// InputStream is = getScriptAsStream(path); -// if (is == null) { -// throw new IllegalStateException("unable to create a stream for path: " + path); -// } -// try { -// execfile(initPhase, is, path); -// } finally { -// is.close(); -// } -// } -// } -// -// /** -// * does not call script.close() -// * @param initPhase True if the script is not registered. Otherwise false. -// * @param script Input stream to the script file -// * @param path Path to the script file -// * @throws Exception -// */ -// static void execfile(boolean initPhase, InputStream script, String path) throws RuntimeException { -// try { -// // exec the code, arbitrary imports are processed -// interpreter.execfile(script, path); -// } catch (PyException e) { -// if (e.match(Py.SystemExit)) { -// PyObject value = e.value; -// if (PyException.isExceptionInstance(e.value)) { -// value = value.__findattr__("code"); -// } -// if (new PyInteger(0).equals(value)) { -// LOG.info("Script invoked sys.exit(0)"); -// return; -// } -// } -// String message = "Python Error. " + e; -// throw new RuntimeException(message, e); -// } -// } -// -// static void setMain(boolean isMain) { -// if (isMain) { -// interpreter.set("__name__", "__main__"); -// } else { -// interpreter.set("__name__", "__lib__"); -// } -// } -// } -// -// /** -// * Gets the Python function object. -// * @param path Path of the jython script file containing the function. -// * @param functionName Name of the function -// * @return a function object -// * @throws IOException -// */ -// public static PyFunction getFunction(String path, String functionName) throws IOException { -// Interpreter.setMain(false); -// Interpreter.init(false, path); -// return (PyFunction) Interpreter.interpreter.get(functionName); -// } -// -// @Override -// protected String getScriptingLang() { -// return "jython"; -// } -// -// @Override -// protected Map getParamsFromVariables() throws IOException { -// PyFrame frame = Py.getFrame(); -// @SuppressWarnings("unchecked") -// List locals = ((PyStringMap) frame.getLocals()).items(); -// Map vars = new HashMap(); -// for (PyTuple item : locals) { -// String key = (String) item.get(0); -// Object obj = item.get(1); -// if (obj != null) { -// String value = item.get(1).toString(); -// vars.put(key, value); -// } -// } -// return vars; -// } -// -// /** -// * File.deleteOnExit(File) does not work for a non-empty directory. This -// * Thread is used to clean up the python.cachedir (if it was a tmp dir -// * created by the Engine) -// */ -// private static class DirDeleter extends Thread { -// private final File dir; -// public DirDeleter(final File file) { -// dir = file; -// } -// @Override -// public void run() { -// try { -// delete(dir); -// } catch (Exception e) { -// LOG.warn("on cleanup", e); -// } -// } -// private static boolean delete(final File file) { -// if (file.isDirectory()) { -// for (File f : file.listFiles()) { -// delete(f); -// } -// } -// return file.delete(); -// } -// } -// -// public static Set registerFunctions(String path, String namespace) -// throws IOException { -// Interpreter.setMain(false); -// Interpreter.init(true, path); -// PythonInterpreter pi = Interpreter.interpreter; -// @SuppressWarnings("unchecked") -// List locals = ((PyStringMap) pi.getLocals()).items(); -// namespace = (namespace == null) ? "" : namespace + NAMESPACE_SEPARATOR; -// Set functionDescs = TUtil.newHashSet(); -// -// for (PyTuple item : locals) { -// String key = (String) item.get(0); -// Object value = item.get(1); -// if (!key.startsWith(JythonConstants.SKIP_TOKEN) && !key.equals(JythonConstants.SCHEMA_FUNCTION) -// && !key.equals(JythonConstants.OUTPUT_TYPE) -// && !key.equals(JythonConstants.OUTPUT_SCHEMA_FUNCTION) -// && (value instanceof PyFunction) -// && (((PyFunction)value).__findattr__(JythonConstants.SCHEMA_FUNCTION)== null)) { -// PyFunction pyFunction = (PyFunction) value; -// -// // Find the pre-defined output schema -// TajoDataTypes.Type returnType; -// PyObject obj = pyFunction.__findattr__(JythonConstants.OUTPUT_TYPE); -// if (obj != null) { -// returnType = pyObjectToType(obj); -// } else { -// // the default return type is the byte array -// returnType = TajoDataTypes.Type.BLOB; -// } -// -// // Parameters have the ANY type. -// int paramNum = ((PyBaseCode) pyFunction.__code__).co_argcount; -// TajoDataTypes.DataType[] paramTypes = new TajoDataTypes.DataType[paramNum]; -// for (int i = 0; i < paramNum; i++) { -// paramTypes[i] = TajoDataTypes.DataType.newBuilder().setType(TajoDataTypes.Type.ANY).build(); -// } -// -// FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, key, -// TajoDataTypes.DataType.newBuilder().setType(returnType).build(), paramTypes); -// FunctionInvocation invocation = new FunctionInvocation(); -// PythonInvocationDesc invocationDesc = new PythonInvocationDesc(key, path); -// invocation.setPython(invocationDesc); -// FunctionSupplement supplement = new FunctionSupplement(); -// functionDescs.add(new FunctionDesc(signature, invocation, supplement)); -// LOG.info("Register scripting UDF: " + namespace + key); -// } -// } -// -// Interpreter.setMain(true); -// return functionDescs; -// } -// -// private static TajoDataTypes.Type pyObjectToType(PyObject obj) { -// return TajoDataTypes.Type.valueOf(pyObjectToTypeStringCand(obj).toUpperCase()); -// } -// -// private static String pyObjectToTypeStringCand(PyObject obj) { -// String[] types = obj.toString().split(","); -// if (types.length > 1) { -// throw new UnsupportedException("Multiple return type is not supported"); -// } -// return types[0].trim(); -// } -//} -// diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java deleted file mode 100644 index fdeee6560d..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/JythonUtils.java +++ /dev/null @@ -1,253 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.python; - -import com.google.common.base.Preconditions; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.datum.AnyDatum; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.DatumFactory; -import org.apache.tajo.exception.UnsupportedException; -import org.apache.tajo.storage.Tuple; -//import org.python.core.*; - -public class JythonUtils { - -// /** -// * Convert a datum to a PyObject. -// * @param v -// * @return -// */ -// public static PyObject datumToPyObject(Datum v) { -// Preconditions.checkArgument(v.type() == TajoDataTypes.Type.ANY); -// Datum actual = ((AnyDatum) v).getActual(); -// switch (actual.type()) { -// case NULL_TYPE: -// return Py.java2py(null); -// case BOOLEAN: -// return Py.java2py(actual.asBool()); -// case UINT1: -// case INT1: -// return Py.java2py(actual.asInt2()); -// case UINT2: -// case INT2: -// return Py.java2py(actual.asInt2()); -// case UINT4: -// case INT4: -// return Py.java2py(actual.asInt4()); -// case UINT8: -// case INT8: -// return Py.java2py(actual.asInt8()); -// case FLOAT4: -// case FLOAT8: -// return Py.java2py(actual.asFloat8()); -// case CHAR: -// case VARCHAR: -// case TEXT: -// return Py.java2py(actual.asChars()); -// case NCHAR: -// case NVARCHAR: -// return Py.java2py(actual.asUnicodeChars()); -// case BLOB: -// return Py.java2py(actual.asByteArray()); -// case INET4: -// return Py.java2py(actual.asByteArray()); -// case INET6: -// return Py.java2py(actual.asByteArray()); -// default: -// throw new UnsupportedException("Unsupported type: " + actual.type()); -// } -// } -// -// /** -// * Convert a Tajo tuple to a PyTuple -// * @param tuple -// * @return -// */ -// public static PyTuple tupleToPyTuple(Tuple tuple) { -// PyObject[] pyTuple = new PyObject[tuple.size()]; -// int i = 0; -// for (Datum v : tuple.getValues()) { -// pyTuple[i++] = datumToPyObject(v); -// } -// return new PyTuple(pyTuple); -// } - - public static Datum objectToDatum(TajoDataTypes.DataType type, Object o) { - switch (type.getType()) { - case BOOLEAN: - return DatumFactory.createBool((Boolean) o); - case INT1: - case INT2: - return DatumFactory.createInt2((Short) o); - case INT4: - return DatumFactory.createInt4((Integer) o); - case INT8: - return DatumFactory.createInt8((Long) o); - case UINT1: - case UINT2: - return DatumFactory.createInt2((Short) o); - case UINT4: - return DatumFactory.createInt4((Integer) o); - case UINT8: - return DatumFactory.createInt8((Long) o); - case FLOAT4: - return DatumFactory.createFloat4((Float) o); - case FLOAT8: - return DatumFactory.createFloat8((Double) o); - case CHAR: - return DatumFactory.createChar((Character) o); - case TEXT: - return DatumFactory.createText((String) o); - case DATE: - return DatumFactory.createDate((Integer) o); - case TIME: - return DatumFactory.createTime((Long) o); - case TIMESTAMP: - return DatumFactory.createTimestamp((Long) o); - case INTERVAL: - return DatumFactory.createInterval((Long) o); - case BLOB: - return DatumFactory.createBlob((byte[]) o); - case INET4: - return DatumFactory.createInet4((Integer) o); - default: - throw new UnsupportedException(type.toString()); - } - } - -// /** -// * Convert a PyObject to a datum. -// * @param object -// * @return -// */ -// public static Datum pyObjectToDatum(PyObject object) { -// if (object instanceof PyLong) { -// return DatumFactory.createInt8((Long) object.__tojava__(Long.class)); -// } else if (object instanceof PyBoolean) { -// return DatumFactory.createBool((Boolean) object.__tojava__(Boolean.class)); -// } else if (object instanceof PyInteger) { -// return DatumFactory.createInt4((Integer) object.__tojava__(Integer.class)); -// } else if (object instanceof PyFloat) { -// // J(P)ython is loosely typed, supports only float type, -// // hence we convert everything to double to save precision -// return DatumFactory.createFloat8((Double) object.__tojava__(Double.class)); -// } else if (object instanceof PyString) { -// return DatumFactory.createText((String) object.__tojava__(String.class)); -// } else if (object instanceof PyNone) { -// return DatumFactory.createNullDatum(); -// } else if (object instanceof PyTuple) { -// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); -// } else if (object instanceof PyList) { -// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); -// } else if (object instanceof PyDictionary) { -// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); -// } else { -// Object javaObj = object.__tojava__(byte[].class); -// if(javaObj instanceof byte[]) { -// return DatumFactory.createBlob((byte[]) javaObj); -// } -// else { -// throw new UnsupportedException("Not supported data type: " + object.getClass().getName()); -// } -// } -// } -// -// /** -// * Convert a pyObject to a datum of the given type. -// * @param object an object will be converted to a datum. -// * @param type target datum type. -// * @return a datum of the given type. -// */ -// public static Datum pyObjectToDatum(PyObject object, TajoDataTypes.Type type) { -// return DatumFactory.cast(pyObjectToDatum(object), CatalogUtil.newSimpleDataType(type), null); -// } - - /** - * Convert the primitive type to the Tajo type. - * @param clazz - * @return - */ - public static TajoDataTypes.Type primitiveTypeToDataType(Class clazz) { - if (clazz.getName().equals(Long.class.getName())) { - return TajoDataTypes.Type.INT8; - } else if (clazz.getName().equals(Boolean.class.getName())) { - return TajoDataTypes.Type.BOOLEAN; - } else if (clazz.getName().equals(Integer.class.getName())) { - return TajoDataTypes.Type.INT4; - } else if (clazz.getName().equals(Float.class.getName())) { - // J(P)ython is loosely typed, supports only float type, - // hence we convert everything to double to save precision - return TajoDataTypes.Type.FLOAT4; - } else if (clazz.getName().equals(Double.class.getName())) { - return TajoDataTypes.Type.FLOAT8; - } else if (clazz.getName().equals(String.class.getName())) { - return TajoDataTypes.Type.TEXT; - } else { - if(clazz.getName().equals(byte[].class.getName())) { - return TajoDataTypes.Type.BLOB; - } - else { - throw new UnsupportedException("Not supported data type: " + clazz.getName()); - } - } - } - - /** - * Convert the Tajo type to the primitive type. - * @param type - * @return - */ - public static Object dataTypeToPrimitiveType(TajoDataTypes.Type type) { - switch (type) { - case BOOLEAN: - return Boolean.class; - case UINT1: - case INT1: - case UINT2: - case INT2: - return Short.class; - case UINT4: - case INT4: - return Integer.class; - case UINT8: - case INT8: - return Long.class; - case FLOAT4: - return Float.class; - case FLOAT8: - return Double.class; - case CHAR: - case VARCHAR: - return Character.class; - case TEXT: - case NCHAR: - case NVARCHAR: - return String.class; - case BLOB: - return Byte[].class; - case INET4: - case INET6: - return Byte[].class; - default: - throw new UnsupportedException("Unsupported type: " + type); - } - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index 09284e0829..e4ade18cd4 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -31,6 +31,7 @@ import org.apache.tajo.util.TUtil; import java.io.*; +import java.net.URI; import java.nio.charset.Charset; import java.util.List; import java.util.Map; @@ -39,22 +40,13 @@ public class PythonScriptEngine extends TajoScriptEngine { - private static final Log log = LogFactory.getLog(PythonScriptEngine.class); + public static final String FILE_EXTENSION = ".py"; + private static final Log LOG = LogFactory.getLog(PythonScriptEngine.class); - public static Set registerFunctions(String path, String namespace) throws IOException { + public static Set registerFunctions(URI path, String namespace) throws IOException { Set functionDescs = TUtil.newHashSet(); - String command = "python"; - String fileName = path.substring(0, path.length() - ".py".length()); - log.debug("Path: " + path + " FileName: " + fileName + " Namespace: " + namespace); -// File f = new File(path); -// -// if (!f.canRead()) { -// throw new IOException("Can't read file: " + path); -// } -// -// FileInputStream fin = new FileInputStream(f); InputStream in = getScriptAsStream(path); List functions = null; try { @@ -62,16 +54,12 @@ public static Set registerFunctions(String path, String namespace) } finally { in.close(); } - namespace = namespace == null ? "" : namespace + NAMESPACE_SEPARATOR; for(FuncInfo funcInfo : functions) { - String alias = namespace + funcInfo.funcName; - log.debug("Registering Function: " + alias); - TajoDataTypes.DataType returnType = CatalogUtil.newSimpleDataType(TajoDataTypes.Type.valueOf(funcInfo.returnType)); FunctionSignature signature = new FunctionSignature(CatalogProtos.FunctionType.UDF, funcInfo.funcName, returnType, createParamTypes(funcInfo.paramNum)); FunctionInvocation invocation = new FunctionInvocation(); - PythonInvocationDesc invocationDesc = new PythonInvocationDesc(funcInfo.funcName, path); + PythonInvocationDesc invocationDesc = new PythonInvocationDesc(funcInfo.funcName, path.getPath()); invocation.setPython(invocationDesc); FunctionSupplement supplement = new FunctionSupplement(); functionDescs.add(new FunctionDesc(signature, invocation, supplement)); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java similarity index 69% rename from tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index c0dc9bf3a9..6f07b8653b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PythonFunctionInvoke2.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -1,5 +1,5 @@ /* - * Lisensed to the Apache Software Foundation (ASF) under one + * 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 @@ -7,7 +7,7 @@ * "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 + * 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, @@ -16,53 +16,53 @@ * limitations under the License. */ -package org.apache.tajo.plan.expr; +package org.apache.tajo.plan.function.python; import com.google.common.base.Charsets; import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.tajo.OverridableConf; +import org.apache.tajo.QueryVars; import org.apache.tajo.catalog.*; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.conf.TajoConf; import org.apache.tajo.datum.Datum; import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.function.PythonInvocationDesc; -import org.apache.tajo.plan.function.python.JythonUtils; -import org.apache.tajo.plan.function.python.ScriptingOutputCapturer; +import org.apache.tajo.plan.function.FunctionInvokeContext; import org.apache.tajo.plan.function.stream.*; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; import java.io.*; -import java.util.Scanner; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; -public class PythonFunctionInvoke2 extends FunctionInvoke { +public class PythonScriptExecutor { - private static final Log LOG = LogFactory.getLog(PythonFunctionInvoke2.class); + private static final Log LOG = LogFactory.getLog(PythonScriptExecutor.class); - private static final String PYTHON_CONTROLLER_JAR_PATH = "/python/controller.py"; //Relative to root of tajo jar. - private static final String PYTHON_TAJO_UTIL_PATH = "/python/tajo_util.py"; //Relative to root of tajo jar. + private static final String PYTHON_ROOT_PATH = "/python"; + private static final String PYTHON_CONTROLLER_JAR_PATH = PYTHON_ROOT_PATH + "/controller.py"; // Relative to root of tajo jar. + private static final String PYTHON_TAJO_UTIL_PATH = PYTHON_ROOT_PATH + "/tajo_util.py"; // Relative to root of tajo jar. + private static final String DEFAULT_LOG_DIR = "/tmp/tajo-" + System.getProperty("user.name") + "/python"; - //Indexes for arguments being passed to external process + // Indexes for arguments being passed to external process private static final int UDF_LANGUAGE = 0; private static final int PATH_TO_CONTROLLER_FILE = 1; - private static final int UDF_FILE_NAME = 2; //Name of file where UDF function is defined - private static final int UDF_FILE_PATH = 3; //Path to directory containing file where UDF function is defined - private static final int UDF_NAME = 4; //Name of UDF function being called. - private static final int PATH_TO_FILE_CACHE = 5; //Directory where required files (like tajo_util) are cached on cluster nodes. - private static final int STD_OUT_OUTPUT_PATH = 6; //File for output from when user writes to standard output. - private static final int STD_ERR_OUTPUT_PATH = 7; //File for output from when user writes to standard error. - private static final int CONTROLLER_LOG_FILE_PATH = 8; //Controller log file logs progress through the controller script not user code. - private static final int IS_ILLUSTRATE = 9; //Controller captures output differently in illustrate vs running. - - private ScriptingOutputCapturer soc; + private static final int UDF_FILE_NAME = 2; // Name of file where UDF function is defined + private static final int UDF_FILE_PATH = 3; // Path to directory containing file where UDF function is defined + private static final int UDF_NAME = 4; // Name of UDF function being called. + private static final int PATH_TO_FILE_CACHE = 5; // Directory where required files (like tajo_util) are cached on cluster nodes. + private static final int STD_OUT_OUTPUT_PATH = 6; // File for output from when user writes to standard output. + private static final int STD_ERR_OUTPUT_PATH = 7; // File for output from when user writes to standard error. + private static final int CONTROLLER_LOG_FILE_PATH = 8; // Controller log file logs progress through the controller script not user code. + private static final int OUT_SCHEMA = 9; // the schema of the output column private Process process; // Handle to the externwlgns1441 - // al process + // all processes private ProcessErrorThread stderrThread; // thread to get process stderr private ProcessInputThread stdinThread; // thread to send input to process private ProcessOutputThread stdoutThread; //thread to read output from process @@ -82,7 +82,7 @@ public class PythonFunctionInvoke2 extends FunctionInvoke { private volatile StreamingUDFException outerrThreadsError; - private OverridableConf queryContext = null; + private FunctionInvokeContext invokeContext = null; private FunctionSignature functionSignature; private PythonInvocationDesc invocationDesc; @@ -93,8 +93,7 @@ public class PythonFunctionInvoke2 extends FunctionInvoke { private CSVLineSerDe lineSerDe = new CSVLineSerDe(); private TableMeta pipeMeta; - public PythonFunctionInvoke2(FunctionDesc functionDesc) { - super(functionDesc); + public PythonScriptExecutor(FunctionDesc functionDesc) { if (!functionDesc.getInvocation().hasPython()) { throw new IllegalStateException("Function type must be python"); } @@ -113,47 +112,51 @@ public PythonFunctionInvoke2(FunctionDesc functionDesc) { pipeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.TEXTFILE); } - @Override - public void init(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) throws IOException { - this.queryContext = queryContext; + public void start(FunctionInvokeContext context) throws IOException { + this.invokeContext = context; this.inputQueue = new ArrayBlockingQueue(1); this.outputQueue = new ArrayBlockingQueue(2); - this.soc = new ScriptingOutputCapturer(queryContext, functionDesc); startUdfController(); createInputHandlers(); setStreams(); startThreads(); + LOG.info("process started"); + } + + public void stop() { + process.destroy(); + LOG.info("process destroyed"); } private StreamingCommand startUdfController() throws IOException { StreamingCommand sc = new StreamingCommand(buildCommand()); - ProcessBuilder processBuilder = StreamingUtil.createProcess(queryContext, sc); + ProcessBuilder processBuilder = StreamingUtil.createProcess(invokeContext.getQueryContext(), sc); process = processBuilder.start(); Runtime.getRuntime().addShutdownHook(new Thread(new ProcessKiller())); - LOG.info("process started"); return sc; } private String[] buildCommand() throws IOException { + OverridableConf queryContext = invokeContext.getQueryContext(); String[] command = new String[10]; -// String standardOutputRootWriteLocation = soc.getStandardOutputRootWriteLocation(); -// String standardOutputRootWriteLocation = System.getProperty("tajo.log.dir"); - // TODO -// String standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; - String standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; + // TODO: support controller logging + String standardOutputRootWriteLocation = ""; + if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { + LOG.warn("Currently, logging is not supported for the python controller."); + standardOutputRootWriteLocation = invokeContext.getQueryContext().get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); + } + standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; String funcName = invocationDesc.getName(); String filePath = invocationDesc.getPath(); - controllerLogFileName = standardOutputRootWriteLocation + funcName + "_python.log"; + controllerLogFileName = standardOutputRootWriteLocation + funcName + "_controller.log"; outFileName = standardOutputRootWriteLocation + funcName + ".out"; errOutFileName = standardOutputRootWriteLocation + funcName + ".err"; - soc.registerOutputLocation(funcName, outFileName); - command[UDF_LANGUAGE] = "python"; command[PATH_TO_CONTROLLER_FILE] = getControllerPath(); int lastSeparator = filePath.lastIndexOf(File.separator) + 1; @@ -165,69 +168,19 @@ private String[] buildCommand() throws IOException { filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; // TODO - String fileCachePath = filePath.substring(0, lastSeparator); -// command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; -// command[PATH_TO_FILE_CACHE] = "'" + "/Users/jihoonson/Projects/tajo/tajo-core/src/test/resources/python/" + "'"; - command[PATH_TO_FILE_CACHE] = "'" + "/home/jihoon/Projects/tajo/tajo-core/src/test/resources/python/" + "'"; + if (!invokeContext.getQueryContext().containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { + throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); + } + String fileCachePath = invokeContext.getQueryContext().get(QueryVars.PYTHON_SCRIPT_CODE_DIR); + command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; command[STD_OUT_OUTPUT_PATH] = outFileName; command[STD_ERR_OUTPUT_PATH] = errOutFileName; command[CONTROLLER_LOG_FILE_PATH] = controllerLogFileName; - command[IS_ILLUSTRATE] = "false"; - -// ensureUserFileAvailable(command, fileCachePath); - - for (String cmd : command) { - LOG.info(cmd); - } + command[OUT_SCHEMA] = outSchema.getColumn(0).getDataType().getType().name().toLowerCase(); return command; } - /** - * Need to make sure the user's file is available. If jar hasn't been - * exploded, just copy the udf file to its path relative to the controller - * file and update file cache path appropriately. - */ - private void ensureUserFileAvailable(String[] command, String fileCachePath) - throws IOException { - - File userUdfFile = new File(fileCachePath + command[UDF_FILE_NAME]); - if (!userUdfFile.exists()) { - String filePath = invocationDesc.getPath(); - String absolutePath = filePath.startsWith("/") ? filePath : "/" + filePath; - absolutePath = absolutePath.replaceAll(":", ""); - String controllerDir = new File(command[PATH_TO_CONTROLLER_FILE]).getParent(); - String userUdfPath = controllerDir + absolutePath + getUserFileExtension(); - userUdfFile = new File(userUdfPath); - userUdfFile.deleteOnExit(); - userUdfFile.getParentFile().mkdirs(); - if (userUdfFile.exists()) { - userUdfFile.delete(); - if (!userUdfFile.createNewFile()) { - throw new IOException("Unable to create file: " + userUdfFile.getAbsolutePath()); - } - } - InputStream udfFileStream = this.getClass().getResourceAsStream( - absolutePath + getUserFileExtension()); - command[PATH_TO_FILE_CACHE] = "\"" + userUdfFile.getParentFile().getAbsolutePath() - + "\""; - - try { - FileUtils.copyInputStreamToFile(udfFileStream, userUdfFile); - } - catch (Exception e) { - throw new IOException("Unable to copy user udf file: " + userUdfFile.getName(), e); - } - finally { - udfFileStream.close(); - } - } - } - - private String getUserFileExtension() { - return ".py"; - } - private void createInputHandlers() { TextLineSerializer serializer = lineSerDe.createSerializer(inSchema, pipeMeta); serializer.init(); @@ -238,17 +191,13 @@ private void createInputHandlers() { } private void setStreams() throws IOException { - stdout = new DataInputStream(new BufferedInputStream(process - .getInputStream())); - outputHandler.bindTo("", stdout, - 0, Long.MAX_VALUE); + stdout = new DataInputStream(new BufferedInputStream(process.getInputStream())); + outputHandler.bindTo(stdout); - stdin = new DataOutputStream(new BufferedOutputStream(process - .getOutputStream())); + stdin = new DataOutputStream(new BufferedOutputStream(process.getOutputStream())); inputHandler.bindTo(stdin); - stderr = new DataInputStream(new BufferedInputStream(process - .getErrorStream())); + stderr = new DataInputStream(new BufferedInputStream(process.getErrorStream())); } private void startThreads() { @@ -297,18 +246,7 @@ private String getControllerPath() throws IOException { return controllerPath; } - @Override - public Datum eval(Tuple tuple) { - return getOutput(tuple); - } - - @Override - public void close() { - process.destroy(); - LOG.info("process destroyed"); - } - - private Datum getOutput(Tuple input) { + public Datum eval(Tuple input) { if (outputQueue == null) { throw new RuntimeException("Process has already been shut down. No way to retrieve output for input: " + input); } @@ -320,14 +258,15 @@ private Datum getOutput(Tuple input) { //We want it to be nothing (since that's what the user wrote). input = new VTuple(0); } - LOG.info("input: " + input); inputQueue.put(input); + LOG.info(inputQueue.size() + ", " + input); } catch (Exception e) { throw new RuntimeException("Failed adding input to inputQueue", e); } Object o = null; try { if (outputQueue != null) { + LOG.info("outputQueue.size(): " + outputQueue.size()); o = outputQueue.take(); if (o == NULL_OBJECT) { o = null; @@ -358,14 +297,11 @@ class ProcessInputThread extends Thread { public void run() { try { - LOG.info("Starting PIT"); while (true) { Tuple inputTuple = inputQueue.take(); - LOG.info("PIT: " + inputTuple); inputHandler.putNext(inputTuple); try { stdin.flush(); - LOG.info("PIT flushed"); } catch(Exception e) { return; } @@ -388,18 +324,23 @@ class ProcessOutputThread extends Thread { } public void run() { - Object o = null; - try{ - LOG.info("Starting POT"); + Object o; + try { o = outputHandler.getNext().get(0); while (o != OutputHandler.END_OF_OUTPUT) { - if (o != null) + if (o != null) { outputQueue.put(o); - else + LOG.info("put " + o + " to outputQueue"); + } + else { outputQueue.put(NULL_OBJECT); + LOG.info("put NULL_OBJECT to outputQueue"); + } o = outputHandler.getNext().get(0); } + } catch (IOException e) { + LOG.warn(e); } catch(Exception e) { if (outputQueue != null) { try { @@ -418,6 +359,7 @@ public void run() { invocationDesc.getName() + " matches the data type being returned.", e); } outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. + LOG.info("put ERROR_OUTPUT to outputQueue"); } catch(InterruptedException ie) { LOG.error(ie); } @@ -433,7 +375,6 @@ public ProcessErrorThread() { public void run() { try { - LOG.info("Starting PET"); Integer lineNumber = null; StringBuffer error = new StringBuffer(); String errInput; @@ -455,6 +396,7 @@ public void run() { outerrThreadsError = new StreamingUDFException("python", error.toString(), lineNumber); if (outputQueue != null) { outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. + LOG.info("put ERROR_OUTPUT to outputQueue"); } if (stderr != null) { stderr.close(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java deleted file mode 100644 index 0db11058b8..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptingOutputCapturer.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Lisensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.python; - -import com.google.common.base.Charsets; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.tajo.OverridableConf; -import org.apache.tajo.catalog.FunctionDesc; -import org.apache.tajo.util.TUtil; - -import java.io.*; -import java.util.Map; -import java.util.UUID; - -public class ScriptingOutputCapturer { - private static Log log = LogFactory.getLog(ScriptingOutputCapturer.class); - - private static Map outputFileNames = TUtil.newHashMap(); - //Unique ID for this run to ensure udf output files aren't corrupted from previous runs. - private static String runId = UUID.randomUUID().toString(); - - //Illustrate will set the static flag telling udf to start capturing its output. It's up to each - //instance to react to it and set its own flag. - private static boolean captureOutput = false; - private boolean instancedCapturingOutput = false; - - private FunctionDesc functionDesc; - private OverridableConf queryContext; - - public ScriptingOutputCapturer(OverridableConf queryContext, FunctionDesc functionDesc) { - this.queryContext = queryContext; - this.functionDesc = functionDesc; - } - - public static void startCapturingOutput() { - ScriptingOutputCapturer.captureOutput = true; - } - - public static Map getUdfOutput() throws IOException { - Map udfFuncNameToOutput = TUtil.newHashMap(); - for (Map.Entry funcToOutputFileName : outputFileNames.entrySet()) { - StringBuffer udfOutput = new StringBuffer(); - FileInputStream fis = new FileInputStream(funcToOutputFileName.getValue()); - Reader fr = new InputStreamReader(fis, Charsets.UTF_8); - BufferedReader br = new BufferedReader(fr); - - try { - String line = br.readLine(); - while (line != null) { - udfOutput.append("\t" + line + "\n"); - line = br.readLine(); - } - } finally { - br.close(); - } - udfFuncNameToOutput.put(funcToOutputFileName.getKey(), udfOutput.toString()); - } - return udfFuncNameToOutput; - } - - public void registerOutputLocation(String functionName, String fileName) { - outputFileNames.put(functionName, fileName); - } - - public static String getRunId() { - return runId; - } - - public static boolean isClassCapturingOutput() { - return ScriptingOutputCapturer.captureOutput; - } - - public boolean isInstanceCapturingOutput() { - return this.instancedCapturingOutput; - } - - public void setInstanceCapturingOutput(boolean instanceCapturingOutput) { - this.instancedCapturingOutput = instanceCapturingOutput; - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 55e4115500..76d3e6a602 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -18,14 +18,12 @@ package org.apache.tajo.plan.function.python; -import org.apache.hadoop.util.Shell; - -import javax.script.ScriptEngine; import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.InputStream; import java.io.IOException; +import java.net.URI; import java.util.Map; public abstract class TajoScriptEngine { @@ -38,8 +36,8 @@ public abstract class TajoScriptEngine { * @return a stream (it is the responsibility of the caller to close it) * @throws IllegalStateException if we could not open a stream */ - protected static InputStream getScriptAsStream(String scriptPath) { - InputStream is; + protected static InputStream getScriptAsStream(URI scriptPath) { + InputStream is = null; File file = new File(scriptPath); if (file.exists()) { try { @@ -47,28 +45,25 @@ protected static InputStream getScriptAsStream(String scriptPath) { } catch (FileNotFoundException e) { throw new IllegalStateException("could not find existing file "+scriptPath, e); } - } else { - if (Shell.WINDOWS && scriptPath.charAt(1)==':') { - scriptPath = scriptPath.charAt(0) + scriptPath.substring(2); - } - // Try system, current and context classloader. - is = ScriptEngine.class.getResourceAsStream(scriptPath); - if (is == null) { - is = getResourceUsingClassLoader(scriptPath, ScriptEngine.class.getClassLoader()); - } - if (is == null) { - is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); - } - if (is == null && !file.isAbsolute()) { - String path = "/" + scriptPath; - is = ScriptEngine.class.getResourceAsStream(path); - if (is == null) { - is = getResourceUsingClassLoader(path, ScriptEngine.class.getClassLoader()); - } - if (is == null) { - is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); - } - } +// } else { +// // Try system, current and context classloader. +// is = TajoScriptEngine.class.getResourceAsStream(scriptPath); +// if (is == null) { +// is = getResourceUsingClassLoader(scriptPath, TajoScriptEngine.class.getClassLoader()); +// } +// if (is == null) { +// is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); +// } +// if (is == null && !file.isAbsolute()) { +// String path = "/" + scriptPath; +// is = TajoScriptEngine.class.getResourceAsStream(path); +// if (is == null) { +// is = getResourceUsingClassLoader(path, TajoScriptEngine.class.getClassLoader()); +// } +// if (is == null) { +// is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); +// } +// } } if (is == null) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java index 277ae45198..f10093195b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufLineReader.java @@ -87,20 +87,16 @@ private void fillBuffer() throws IOException { boolean release = true; try { int readBytes = tailBytes; -// for (; ; ) { - int localReadBytes = buffer.writeBytes(channel, this.bufferSize - readBytes); - if (localReadBytes < 0) { - if (buffer.isWritable()) { - //if read bytes is less than the buffer capacity, there is no more bytes in the channel - eof = true; - } -// break; + // read only once + int localReadBytes = buffer.writeBytes(channel, this.bufferSize - readBytes); + if (localReadBytes < 0) { + if (buffer.isWritable()) { + //if read bytes is less than the buffer capacity, there is no more bytes in the channel + eof = true; } - readBytes += localReadBytes; -// if (readBytes == bufferSize) { -// break; -// } -// } + } + readBytes += localReadBytes; + this.readBytes += (readBytes - tailBytes); release = false; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java index 2681105adb..e9d957ecc9 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -54,12 +54,6 @@ public enum InputType {SYNCHRONOUS, ASYNCHRONOUS} // flag to mark if close() has already been called protected boolean alreadyClosed = false; - /** - * Get the handled InputType - * @return the handled InputType - */ - public abstract InputType getInputType(); - /** * Send the given input Tuple to the managed executable. * diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java index ac5813a18b..3003de3fd9 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java @@ -20,6 +20,7 @@ import com.google.common.base.Charsets; import io.netty.buffer.ByteBuf; +import org.apache.commons.codec.binary.Base64; import org.apache.tajo.OverridableConf; import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; @@ -54,18 +55,14 @@ public enum OutputType {SYNCHRONOUS, ASYNCHRONOUS} private InputStream istream; + private final ByteBuf buf = BufferPool.directBuffer(DEFAULT_BUFFER); + //Both of these ignore the trailing \n. So if the //default delimiter is "\n" recordDelimStr is "". private String recordDelimStr = null; private int recordDelimLength = 0; private Tuple tuple = new VTuple(1); - /** - * Get the handled OutputType. - * @return the handled OutputType - */ - public abstract OutputType getOutputType(); - // flag to mark if close() has already been called protected boolean alreadyClosed = false; @@ -77,12 +74,9 @@ public enum OutputType {SYNCHRONOUS, ASYNCHRONOUS} * of the managed process * @throws IOException */ - public void bindTo(String fileName, InputStream is, - long offset, long end) throws IOException { + public void bindTo(InputStream is) throws IOException { this.istream = is; this.in = new ByteBufLineReader(new ByteBufInputChannel(istream)); - - // TODO } /** @@ -100,7 +94,6 @@ public Tuple getNext() throws IOException { if (!readValue()) { return null; } - ByteBuf buf = BufferPool.directBuffer(DEFAULT_BUFFER); buf.writeBytes(currValue.getBytes()); try { deserializer.deserialize(buf, tuple); @@ -118,7 +111,6 @@ private boolean readValue() throws IOException { while(!isEndOfRow()) { //Need to add back the newline character we ate. -// currValue.append(new byte[] {'\n'}, 0, 1); currValue += '\n'; byte[] lineBytes = readNextLine(); @@ -126,7 +118,6 @@ private boolean readValue() throws IOException { //We have no more input, so just break; break; } -// currValue.append(lineBytes, 0, lineBytes.length); currValue += new String(lineBytes); } @@ -139,7 +130,6 @@ private boolean readValue() throws IOException { } private byte[] readNextLine() throws IOException { -// Text line = new Text(); String line = in.readLine(); if (line == null) { return null; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java index 1341785dc0..44d22c2128 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java @@ -18,8 +18,6 @@ package org.apache.tajo.plan.function.stream; -import org.apache.tajo.util.TUtil; - import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -53,12 +51,6 @@ public class StreamingCommand implements Serializable, Cloneable { */ public enum Handle {INPUT, OUTPUT} - /** - * Map from the the stdin/stdout/stderr handles to their specifications - */ - Map> handleSpecs = - new TreeMap>(); - // Should the stderr of the process be persisted? boolean persistStderr = false; @@ -159,91 +151,6 @@ public void addPathToShip(String path) throws IOException { shipSpec.add(path); } - /** - * Attach a {@link HandleSpec} to a given {@link Handle} - * @param handle Handle to which the specification is to - * be attached. - * @param handleSpec HandleSpec for the given handle. - */ - public void addHandleSpec(Handle handle, HandleSpec handleSpec) { - List handleSpecList = handleSpecs.get(handle); - - if (handleSpecList == null) { - handleSpecList = new LinkedList(); - handleSpecs.put(handle, handleSpecList); - } - - handleSpecList.add(handleSpec); - } - - /** - * Set the input specification for the StreamingCommand. - * - * @param spec input specification - */ - public void setInputSpec(HandleSpec spec) { - List inputSpecs = getHandleSpecs(Handle.INPUT); - if (inputSpecs == null || inputSpecs.size() == 0) { - addHandleSpec(Handle.INPUT, spec); - } else { - inputSpecs.set(0, spec); - } - } - -// /** -// * Get the input specification of the StreamingCommand. -// * -// * @return input specification of the StreamingCommand -// */ -// public HandleSpec getInputSpec() { -// List inputSpecs = getHandleSpecs(Handle.INPUT); -// if (inputSpecs == null || inputSpecs.size() == 0) { -// addHandleSpec(Handle.INPUT, new HandleSpec("stdin", PigStreaming.class.getName())); -// } -// return getHandleSpecs(Handle.INPUT).get(0); -// } - - /** - * Set the specification for the primary output of the - * StreamingCommand. - * - * @param spec specification for the primary output of the - * StreamingCommand - */ - public void setOutputSpec(HandleSpec spec) { - List outputSpecs = getHandleSpecs(Handle.OUTPUT); - if (outputSpecs == null || outputSpecs.size() == 0) { - addHandleSpec(Handle.OUTPUT, spec); - } else { - outputSpecs.set(0, spec); - } - } -// -// /** -// * Get the specification of the primary output of the -// * StreamingCommand. -// * -// * @return specification of the primary output of the -// * StreamingCommand -// */ -// public HandleSpec getOutputSpec() { -// List outputSpecs = getHandleSpecs(Handle.OUTPUT); -// if (outputSpecs == null || outputSpecs.size() == 0) { -// addHandleSpec(Handle.OUTPUT, new HandleSpec("stdout", PigStreaming.class.getName())); -// } -// return getHandleSpecs(Handle.OUTPUT).get(0); -// } - - /** - * Get specifications for the given Handle. - * - * @param handle Handle of the stream - * @return specification for the given Handle - */ - public List getHandleSpecs(Handle handle) { - return handleSpecs.get(handle); - } - /** * Should the stderr of the managed process be persisted? * @@ -324,17 +231,6 @@ public boolean getShipFiles() { return shipFiles; } -// public String toString() { -// StringBuffer sb = new StringBuffer(); -// for (String arg : getCommandArgs()) { -// sb.append(arg); -// sb.append(" "); -// } -// sb.append("(" + getInputSpec().toString() + "/"+getOutputSpec() + ")"); -// -// return sb.toString(); -// } - public Object clone() { try { StreamingCommand clone = (StreamingCommand)super.clone(); @@ -342,136 +238,10 @@ public Object clone() { clone.shipSpec = new ArrayList(shipSpec); clone.cacheSpec = new ArrayList(cacheSpec); - clone.handleSpecs = new HashMap>(); - for (Map.Entry> e : handleSpecs.entrySet()) { - List values = new ArrayList(); - for (HandleSpec spec : e.getValue()) { - values.add((HandleSpec)spec.clone()); - } - clone.handleSpecs.put(e.getKey(), values); - } - return clone; } catch (CloneNotSupportedException cnse) { // Shouldn't happen since we do implement Clonable throw new InternalError(cnse.toString()); } } - - - /** - * Specification about the usage of the {@link Handle} to communicate - * with the external process. - * - * It specifies the stream-handle which can be one of stdin/ - * stdout/stderr or a named file and also the - * serializer/deserializer specification to be used to read/write data - * to/from the stream. - */ - public static class HandleSpec - implements Comparable, Serializable, Cloneable { - private static final long serialVersionUID = 1L; - - String name; -// String spec; - -// /** -// * Create a new {@link HandleSpec} with a given name using the default -// * {@link PigStorage} serializer/deserializer. -// * -// * @param handleName name of the handle (one of stdin, -// * stdout or a file-path) -// */ -// public HandleSpec(String handleName) { -// this(handleName, PigStreaming.class.getName()); -// } - -// /** -// * Create a new {@link HandleSpec} with a given name using the default -// * {@link PigStorage} serializer/deserializer. -// * -// * @param handleName name of the handle (one of stdin, -// * stdout or a file-path) -// * @param spec serializer/deserializer spec -// */ -// public HandleSpec(String handleName, String spec) { -// this.name = handleName; -// this.spec = spec; -// } - - public HandleSpec(String handleName) { - this.name = handleName; - } - - public int compareTo(HandleSpec o) { - return this.name.compareTo(o.name); - } - - public String toString() { -// return name + "-" + spec; - return name; - } - - /** - * Get the name of the HandleSpec. - * - * @return the name of the HandleSpec (one of - * stdin, stdout or a file-path) - */ - public String getName() { - return name; - } - - /** - * Set the name of the HandleSpec. - * - * @param name name of the HandleSpec (one of - * stdin, stdout or a file-path) - */ - public void setName(String name) { - this.name = name; - } - -// /** -// * Get the serializer/deserializer spec of the HandleSpec. -// * -// * @return the serializer/deserializer spec of the -// * HandleSpec -// */ -// public String getSpec() { -// return spec; -// } -// -// /** -// * Set the serializer/deserializer spec of the HandleSpec. -// * -// * @param spec the serializer/deserializer spec of the -// * HandleSpec -// */ -// public void setSpec(String spec) { -// this.spec = spec; -// } - - public boolean equals(Object obj) { - if (obj instanceof HandleSpec){ - HandleSpec other = (HandleSpec)obj; -// return (other != null && name.equals(other.name) && spec.equals(other.spec)); - return TUtil.checkEquals(name, other.name); - } else - return false; - } - - public int hashCode() { - return name.hashCode(); - } - - public Object clone() { - try { - return super.clone(); - } catch (CloneNotSupportedException cnse) { - // Shouldn't happen since we do implement Clonable - throw new InternalError(cnse.toString()); - } - } - } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java index dcc3c25835..e7354f8672 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java @@ -26,11 +26,6 @@ public StreamingUDFInputHandler(TextLineSerializer serializer) { this.serializer = serializer; } - @Override - public InputType getInputType() { - return InputType.SYNCHRONOUS; - } - @Override public synchronized void close(Process process) throws IOException { try { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java index 75eb3b151b..5b8e2ecaff 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java @@ -28,9 +28,4 @@ public StreamingUDFOutputHandler(TextLineDeserializer deserializer) { protected byte[] getRecordDelimiter() { return ",".getBytes(); } - - @Override - public OutputType getOutputType() { - return OutputType.SYNCHRONOUS; - } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUtil.java similarity index 89% rename from tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java rename to tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUtil.java index fb0e65286b..607353723c 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/StreamingUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUtil.java @@ -16,12 +16,11 @@ * limitations under the License. */ -package org.apache.tajo.plan.expr; +package org.apache.tajo.plan.function.stream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.tajo.OverridableConf; -import org.apache.tajo.plan.function.stream.StreamingCommand; import java.io.File; import java.util.ArrayList; @@ -82,7 +81,6 @@ public static ProcessBuilder createProcess(OverridableConf queryContext, Streami private static void setupEnvironment(OverridableConf queryContext, ProcessBuilder pb) { String separator = ":"; Map env = pb.environment(); -// addJobConfToEnvironment(queryContext, env); // Add the current-working-directory to the $PATH File dir = pb.directory(); @@ -97,11 +95,4 @@ private static void setupEnvironment(OverridableConf queryContext, ProcessBuilde } env.put(PATH, envPath); } - - private static void envPut(Map env, String name, String value) { - if (LOG.isDebugEnabled()) { - LOG.debug("Add env entry:" + name + "=" + value); - } - env.put(name, value); - } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java index b809820877..8d7dff5b1b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java @@ -244,8 +244,8 @@ public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullC case BLOB: { byte[] bytes = new byte[buf.readableBytes()]; buf.readBytes(bytes); -// datum = DatumFactory.createBlob(Base64.decodeBase64(bytes)); - datum = DatumFactory.createBlob(bytes); + datum = DatumFactory.createBlob(Base64.decodeBase64(bytes)); +// datum = DatumFactory.createBlob(bytes); break; } default: From d30f38a92de751a97de2abfae6d2e37f2b90e688 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 7 Apr 2015 21:07:20 +0900 Subject: [PATCH 31/55] TAJO-1344_3 --- .../apache/tajo/plan/function/PythonFunctionInvoke.java | 8 +++++++- .../tajo/plan/function/python/PythonScriptExecutor.java | 7 ------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index 5721dc3541..d111f69252 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -28,6 +28,7 @@ public class PythonFunctionInvoke extends FunctionInvoke { private PythonScriptExecutor scriptExecutor; + private FunctionInvokeContext context; public PythonFunctionInvoke(FunctionDesc functionDesc) { super(functionDesc); @@ -36,11 +37,16 @@ public PythonFunctionInvoke(FunctionDesc functionDesc) { @Override public void init(FunctionInvokeContext context) throws IOException { - scriptExecutor.start(context); + this.context = context; } @Override public Datum eval(Tuple tuple) { + try { + scriptExecutor.start(context); + } catch (IOException e) { + throw new RuntimeException(e); + } return scriptExecutor.eval(tuple); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index 6f07b8653b..1588a3559c 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -133,7 +133,6 @@ private StreamingCommand startUdfController() throws IOException { ProcessBuilder processBuilder = StreamingUtil.createProcess(invokeContext.getQueryContext(), sc); process = processBuilder.start(); - Runtime.getRuntime().addShutdownHook(new Thread(new ProcessKiller())); return sc; } @@ -409,10 +408,4 @@ public void run() { } } } - - public class ProcessKiller implements Runnable { - public void run() { - process.destroy(); - } - } } From 751706e5b694e3cbeff24723d58d83a9473af061 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Tue, 7 Apr 2015 23:51:49 +0900 Subject: [PATCH 32/55] TAJO-1344_3 --- .../tajo/engine/function/TestPythonFunctions.java | 6 ++++++ .../apache/tajo/engine/query/TestGroupByQuery.java | 5 +++++ .../org/apache/tajo/engine/query/TestSelectQuery.java | 11 +++++------ .../plan/function/python/PythonScriptExecutor.java | 5 ++--- 4 files changed, 18 insertions(+), 9 deletions(-) diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java index 09e26b73a9..2435ad7978 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -35,4 +35,10 @@ public void test() throws IOException { testSimpleEval("select percent(386, 1000)", new String[]{"38.6"}); testSimpleEval("select concat4('Tajo', 'is', 'awesome', '!')", new String[]{"Tajo is awesome !"}); } + + @Test + public void testNestedFunctions() throws IOException { + testSimpleEval("select sum_py(3, return_one())", new String[]{"4"}); + testSimpleEval("select concat_py(helloworld())", new String[]{"Hello, WorldHello, World"}); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java index d1756e1a2e..71d3d81961 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java @@ -799,4 +799,9 @@ public final void testGroupbyWithLimit3() throws Exception { assertResultSet(res); cleanupQuery(res); } + + @Test + public final void testGroupbyWithPythonFunc() throws Exception { + executeString("select count(*) from nation where sum_py(n_nationkey, 1) > 2 group by n_regionkey"); + } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 219bb7962e..5f2167f40b 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -716,16 +716,15 @@ public void testSelectPythonFuncs() throws Exception { cleanupQuery(res); } - @Test - public void testSelectPythonFuncs2() throws Exception { - executeString("select sum_py(n_nationkey, n_regionkey) as sum\n" + - "from nation where n_nationkey < 5"); - } - @Test public void testSelectWithPredicateOnPythonFunc() throws Exception { ResultSet res = executeQuery(); assertResultSet(res); cleanupQuery(res); } + + @Test + public void testNestedPythonFunction() throws Exception { + executeString("select * from nation where sum_py(n_regionkey, return_one()) > 2"); + } } \ No newline at end of file diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index 1588a3559c..7dbde890d1 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -147,6 +147,7 @@ private String[] buildCommand() throws IOException { standardOutputRootWriteLocation = invokeContext.getQueryContext().get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); } standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; +// standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; String funcName = invocationDesc.getName(); @@ -162,9 +163,7 @@ private String[] buildCommand() throws IOException { String fileName = filePath.substring(lastSeparator); fileName = fileName.endsWith(".py") ? fileName.substring(0, fileName.length()-3) : fileName; command[UDF_FILE_NAME] = fileName; - command[UDF_FILE_PATH] = lastSeparator <= 0 ? - "." : - filePath.substring(0, lastSeparator - 1); + command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; // TODO if (!invokeContext.getQueryContext().containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { From 01cff3d8774fc5cb815e5411991b2c9e7a9a41c8 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 00:01:37 +0900 Subject: [PATCH 33/55] TAJO-1343_3 --- .../java/org/apache/tajo/plan/function/stream/BufferPool.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java index 728a4218a9..0b3c625a92 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/BufferPool.java @@ -44,7 +44,7 @@ public static long maxDirectMemory() { } - public synchronized static ByteBuf directBuffer(int size) { + public static ByteBuf directBuffer(int size) { return allocator.directBuffer(size); } From a481cc7653026ba796bb056953eb7268cfc5d79e Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 03:01:35 +0900 Subject: [PATCH 34/55] TAJO-1344_3 --- .../src/main/resources/python/controller.py | 15 +++------- .../tajo/engine/query/TestGroupByQuery.java | 11 ++++++- .../tajo/engine/query/TestSelectQuery.java | 4 ++- .../testGroupbyWithPythonFunc.sql | 1 + .../testGroupbyWithPythonFunc2.sql | 1 + .../testNestedPythonFunction.sql | 1 + .../testGroupbyWithPythonFunc.result | 7 +++++ .../testGroupbyWithPythonFunc2.result | 7 +++++ .../testNestedPythonFunction.result | 7 +++++ .../org/apache/tajo/plan/expr/CastEval.java | 19 ++++++++++-- .../tajo/plan/expr/GeneralFunctionEval.java | 29 +++++++++---------- .../tajo/plan/function/FunctionInvoke.java | 8 +++++ .../plan/function/FunctionInvokeContext.java | 16 ++++++++++ .../function/LegacyScalarFunctionInvoke.java | 10 +++++++ .../plan/function/PythonFunctionInvoke.java | 22 ++++++++++++-- .../function/python/PythonScriptExecutor.java | 12 +------- 16 files changed, 126 insertions(+), 44 deletions(-) create mode 100644 tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc.sql create mode 100644 tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc2.sql create mode 100644 tajo-core/src/test/resources/queries/TestSelectQuery/testNestedPythonFunction.sql create mode 100644 tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc.result create mode 100644 tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc2.result create mode 100644 tajo-core/src/test/resources/results/TestSelectQuery/testNestedPythonFunction.result diff --git a/tajo-core/src/main/resources/python/controller.py b/tajo-core/src/main/resources/python/controller.py index e64aa3ee88..7be02c458e 100644 --- a/tajo-core/src/main/resources/python/controller.py +++ b/tajo-core/src/main/resources/python/controller.py @@ -95,12 +95,11 @@ def main(self, sys.path.append(cache_path) sys.path.append('.') - logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) - logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the " - "INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") + # logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) + # logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the " + # "INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") input_str = self.get_next_input() - logging.info('1: ' + input_str) try: func = __import__(module_name, globals(), locals(), [func_name], -1).__dict__[func_name] @@ -109,20 +108,18 @@ def main(self, write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-1) - logging.info('2: ') # if udf_logging.udf_log_level != logging.DEBUG: # #Only log output for illustrate after we get the flag to capture output. # sys.stdout = open(os.devnull, 'w') # else: # sys.stdout = self.log_stream - should_log = True + should_log = False log_message = logging.info if udf_logging.udf_log_level == logging.DEBUG: should_log = True log_message = logging.debug - logging.info('3: ') while input_str != END_OF_STREAM: try: try: @@ -169,7 +166,6 @@ def get_next_input(self): # log_stream = self.log_stream input_str = input_stream.readline() - logging.info('get_next_input1: ' + input_str) while input_str.endswith(END_RECORD_DELIM) == False: line = input_stream.readline() @@ -178,8 +174,6 @@ def get_next_input(self): break input_str += line - logging.info('get_next_input2: ' + input_str) - if input_str == '': return END_OF_STREAM @@ -191,7 +185,6 @@ def get_next_input(self): if input_str == END_OF_STREAM: return input_str - logging.info('get_next_input3: ' + input_str) return input_str[:-END_RECORD_DELIM_LENGTH] def close_controller(self, exit_code): diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java index 71d3d81961..15a1c9f79d 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestGroupByQuery.java @@ -802,6 +802,15 @@ public final void testGroupbyWithLimit3() throws Exception { @Test public final void testGroupbyWithPythonFunc() throws Exception { - executeString("select count(*) from nation where sum_py(n_nationkey, 1) > 2 group by n_regionkey"); + ResultSet res = executeQuery(); + assertResultSet(res); + cleanupQuery(res); + } + + @Test + public final void testGroupbyWithPythonFunc2() throws Exception { + ResultSet res = executeQuery(); + assertResultSet(res); + cleanupQuery(res); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java index 5f2167f40b..fc1ed3d24c 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/query/TestSelectQuery.java @@ -725,6 +725,8 @@ public void testSelectWithPredicateOnPythonFunc() throws Exception { @Test public void testNestedPythonFunction() throws Exception { - executeString("select * from nation where sum_py(n_regionkey, return_one()) > 2"); + ResultSet res = executeQuery(); + assertResultSet(res); + cleanupQuery(res); } } \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc.sql b/tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc.sql new file mode 100644 index 0000000000..888552a984 --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc.sql @@ -0,0 +1 @@ +select count(*) from nation where sum_py(n_nationkey, 1) > 2 group by n_regionkey \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc2.sql b/tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc2.sql new file mode 100644 index 0000000000..bcfce13d86 --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestGroupByQuery/testGroupbyWithPythonFunc2.sql @@ -0,0 +1 @@ +select n_regionkey, count(*) as cnt from nation group by n_regionkey having percent(cnt, 25) > 10 \ No newline at end of file diff --git a/tajo-core/src/test/resources/queries/TestSelectQuery/testNestedPythonFunction.sql b/tajo-core/src/test/resources/queries/TestSelectQuery/testNestedPythonFunction.sql new file mode 100644 index 0000000000..75b33ae326 --- /dev/null +++ b/tajo-core/src/test/resources/queries/TestSelectQuery/testNestedPythonFunction.sql @@ -0,0 +1 @@ +select * from nation where sum_py(n_regionkey, return_one()) < 2 \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc.result new file mode 100644 index 0000000000..2a5fb8a624 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc.result @@ -0,0 +1,7 @@ +?count +------------------------------- +4 +5 +5 +5 +4 \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc2.result b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc2.result new file mode 100644 index 0000000000..08561fd02d --- /dev/null +++ b/tajo-core/src/test/resources/results/TestGroupByQuery/testGroupbyWithPythonFunc2.result @@ -0,0 +1,7 @@ +n_regionkey,cnt +------------------------------- +0,5 +3,5 +4,5 +2,5 +1,5 \ No newline at end of file diff --git a/tajo-core/src/test/resources/results/TestSelectQuery/testNestedPythonFunction.result b/tajo-core/src/test/resources/results/TestSelectQuery/testNestedPythonFunction.result new file mode 100644 index 0000000000..899f0340c2 --- /dev/null +++ b/tajo-core/src/test/resources/results/TestSelectQuery/testNestedPythonFunction.result @@ -0,0 +1,7 @@ +n_nationkey,n_name,n_regionkey,n_comment +------------------------------- +0,ALGERIA,0, haggle. carefully final deposits detect slyly agai +5,ETHIOPIA,0,ven packages wake quickly. regu +14,KENYA,0, pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t +15,MOROCCO,0,rns. blithely bold courts among the closely regular packages use furiously bold platelets? +16,MOZAMBIQUE,0,s. ironic, unusual asymptotes wake blithely r \ No newline at end of file diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java index f174b79973..64ccc49884 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java @@ -97,11 +97,24 @@ public boolean equals(Object obj) { boolean valid = obj != null && obj instanceof CastEval; if (valid) { CastEval another = (CastEval) obj; - return child.equals(another.child) && - target.equals(another.target) && - TUtil.checkEquals(timezone, another.timezone); + boolean b1 = child.equals(another.child); + boolean b2 = target.equals(another.target); + boolean b3 = TUtil.checkEquals(timezone, another.timezone); + return b1 && b2 && b3; } else { return false; } } + + @Override + public Object clone() throws CloneNotSupportedException { + CastEval clone = (CastEval) super.clone(); + if (target != null) { + clone.target = target; + } + if (timezone != null) { + clone.timezone = timezone; + } + return clone; + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 7c17982fc4..97573fa36d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -33,7 +33,7 @@ import java.util.Map; public class GeneralFunctionEval extends FunctionEval { - @Expose protected FunctionInvoke funcInvoke; + protected FunctionInvoke funcInvoke; @Expose protected FunctionInvokeContext invokeContext; public GeneralFunctionEval(OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) @@ -63,21 +63,20 @@ public Datum eval(Tuple tuple) { return res; } - @Override - public boolean equals(Object obj) { - if (obj instanceof GeneralFunctionEval) { - GeneralFunctionEval other = (GeneralFunctionEval) obj; - return super.equals(other) && - TUtil.checkEquals(funcInvoke, other.funcInvoke); - } - - return false; - } +// @Override +// public boolean equals(Object obj) { +// if (obj instanceof GeneralFunctionEval) { +// GeneralFunctionEval other = (GeneralFunctionEval) obj; +// return super.equals(other); +// } +// +// return false; +// } - @Override - public int hashCode() { - return Objects.hashCode(funcDesc, funcInvoke); - } +// @Override +// public int hashCode() { +// return Objects.hashCode(funcDesc, funcInvoke); +// } @Override public Object clone() throws CloneNotSupportedException { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java index 609c9102e7..c419791d30 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java @@ -31,6 +31,10 @@ public abstract class FunctionInvoke implements Closeable { @Expose protected FunctionDesc functionDesc; + public FunctionInvoke() { + + } + public FunctionInvoke(FunctionDesc functionDesc) { this.functionDesc = functionDesc; } @@ -45,6 +49,10 @@ public static FunctionInvoke newInstance(FunctionDesc desc) throws InternalExcep } } + public void setFunctionDesc(FunctionDesc functionDesc) throws InternalException { + this.functionDesc = functionDesc; + } + public abstract void init(FunctionInvokeContext context) throws IOException; /** diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java index 0e2c3002b5..095ae057cc 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java @@ -18,10 +18,12 @@ package org.apache.tajo.plan.function; +import com.google.common.base.Objects; import org.apache.tajo.OverridableConf; import org.apache.tajo.plan.expr.FunctionEval; import org.apache.tajo.util.KeyValueSet; +import java.util.Arrays; import java.util.Map; public class FunctionInvokeContext { @@ -40,4 +42,18 @@ public OverridableConf getQueryContext() { public FunctionEval.ParamType[] getParamTypes() { return paramTypes; } + + @Override + public int hashCode() { + return Objects.hashCode(queryContext, Arrays.hashCode(paramTypes)); + } + + @Override + public boolean equals(Object o) { + if (o instanceof FunctionInvokeContext) { + FunctionInvokeContext other = (FunctionInvokeContext) o; + return queryContext.equals(other.queryContext) && Arrays.equals(paramTypes, other.paramTypes); + } + return false; + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java index a1d2ad6d05..1b97aaa052 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java @@ -28,11 +28,21 @@ public class LegacyScalarFunctionInvoke extends FunctionInvoke { @Expose private GeneralFunction function; + public LegacyScalarFunctionInvoke() { + + } + public LegacyScalarFunctionInvoke(FunctionDesc funcDesc) throws InternalException { super(funcDesc); function = (GeneralFunction) funcDesc.newInstance(); } + @Override + public void setFunctionDesc(FunctionDesc desc) throws InternalException { + super.setFunctionDesc(desc); + function = (GeneralFunction) functionDesc.newInstance(); + } + @Override public void init(FunctionInvokeContext context) { function.init(context.getQueryContext(), context.getParamTypes()); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index d111f69252..5c9178f19f 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -25,18 +25,22 @@ import java.io.IOException; -public class PythonFunctionInvoke extends FunctionInvoke { +public class PythonFunctionInvoke extends FunctionInvoke implements Cloneable { private PythonScriptExecutor scriptExecutor; private FunctionInvokeContext context; + public PythonFunctionInvoke() { + + } + public PythonFunctionInvoke(FunctionDesc functionDesc) { super(functionDesc); - scriptExecutor = new PythonScriptExecutor(functionDesc); } @Override public void init(FunctionInvokeContext context) throws IOException { + this.scriptExecutor = new PythonScriptExecutor(functionDesc); this.context = context; } @@ -54,4 +58,18 @@ public Datum eval(Tuple tuple) { public void close() { scriptExecutor.stop(); } + +// @Override +// public Object clone() throws CloneNotSupportedException { +// PythonFunctionInvoke clone = (PythonFunctionInvoke) super.clone(); +// if (context != null) { +// clone.context = context; +// } +// return clone; +// } +// +// @Override +// public int hashCode() { +// return Objects.hashCode(super.hashCode(), context); +// } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index 7dbde890d1..b06e7272da 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -120,12 +120,10 @@ public void start(FunctionInvokeContext context) throws IOException { createInputHandlers(); setStreams(); startThreads(); - LOG.info("process started"); } public void stop() { process.destroy(); - LOG.info("process destroyed"); } private StreamingCommand startUdfController() throws IOException { @@ -146,8 +144,6 @@ private String[] buildCommand() throws IOException { LOG.warn("Currently, logging is not supported for the python controller."); standardOutputRootWriteLocation = invokeContext.getQueryContext().get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); } - standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; -// standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; String funcName = invocationDesc.getName(); @@ -257,14 +253,12 @@ public Datum eval(Tuple input) { input = new VTuple(0); } inputQueue.put(input); - LOG.info(inputQueue.size() + ", " + input); } catch (Exception e) { throw new RuntimeException("Failed adding input to inputQueue", e); } Object o = null; try { if (outputQueue != null) { - LOG.info("outputQueue.size(): " + outputQueue.size()); o = outputQueue.take(); if (o == NULL_OBJECT) { o = null; @@ -329,16 +323,14 @@ public void run() { while (o != OutputHandler.END_OF_OUTPUT) { if (o != null) { outputQueue.put(o); - LOG.info("put " + o + " to outputQueue"); } else { outputQueue.put(NULL_OBJECT); - LOG.info("put NULL_OBJECT to outputQueue"); } o = outputHandler.getNext().get(0); } } catch (IOException e) { - LOG.warn(e); + // EOF } catch(Exception e) { if (outputQueue != null) { try { @@ -357,7 +349,6 @@ public void run() { invocationDesc.getName() + " matches the data type being returned.", e); } outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. - LOG.info("put ERROR_OUTPUT to outputQueue"); } catch(InterruptedException ie) { LOG.error(ie); } @@ -394,7 +385,6 @@ public void run() { outerrThreadsError = new StreamingUDFException("python", error.toString(), lineNumber); if (outputQueue != null) { outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. - LOG.info("put ERROR_OUTPUT to outputQueue"); } if (stderr != null) { stderr.close(); From 687fbc3e7b24e171ab3a0724aa8abb0bd086f8b6 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 03:12:00 +0900 Subject: [PATCH 35/55] TAJO-1344_3 --- .../apache/tajo/storage/TestDelimitedTextFile.java | 2 +- .../TestDelimitedTextFile/testErrorTolerance1.json | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java index e2d5481a55..2db916afaf 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/TestDelimitedTextFile.java @@ -68,7 +68,7 @@ public class TestDelimitedTextFile { DatumFactory.createFloat4(77.9f), // 5 DatumFactory.createFloat8(271.9d), // 6 DatumFactory.createText("hyunsik"), // 7 - DatumFactory.createBlob("hyunsik"),// 8 + DatumFactory.createBlob("hyunsik".getBytes()),// 8 DatumFactory.createInet4("192.168.0.1"), // 9 }); } diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json index 739dfe7f5f..d3aee3396e 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json +++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestDelimitedTextFile/testErrorTolerance1.json @@ -1,6 +1,6 @@ -{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"} -{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1" -{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"} -{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1" -{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1" -{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"} \ No newline at end of file +{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "aHl1bnNpaw==", "col10": "192.168.0.1"} +{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "aHl1bnNpaw==", "col10": "192.168.0.1" +{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "aHl1bnNpaw==", "col10": "192.168.0.1"} +{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "aHl1bnNpaw==", "col10": "192.168.0.1" +{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "aHl1bnNpaw==", "col10": "192.168.0.1" +{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "aHl1bnNpaw==", "col10": "192.168.0.1"} \ No newline at end of file From aee4cd80817c3b09ab242b28c2f8d95d6757ed50 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 03:15:57 +0900 Subject: [PATCH 36/55] TAJO-1344_3 --- .../test/java/org/apache/tajo/storage/json/TestJsonSerDe.java | 2 +- .../test/resources/dataset/TestJsonSerDe/testVariousType.json | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java index 243807734d..70282d9a9b 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java +++ b/tajo-storage/tajo-storage-hdfs/src/test/java/org/apache/tajo/storage/json/TestJsonSerDe.java @@ -91,7 +91,7 @@ public void testVarioutType() throws IOException { DatumFactory.createFloat4(77.9f), // 5 DatumFactory.createFloat8(271.9d), // 6 DatumFactory.createText("hyunsik"), // 7 - DatumFactory.createBlob("hyunsik"), // 8 + DatumFactory.createBlob("hyunsik".getBytes()), // 8 DatumFactory.createInet4("192.168.0.1"), // 9 NullDatum.get(), // 10 }); diff --git a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json index 8ee3408f04..ec31982498 100644 --- a/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json +++ b/tajo-storage/tajo-storage-hdfs/src/test/resources/dataset/TestJsonSerDe/testVariousType.json @@ -1 +1 @@ -{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "hyunsik", "col10": "192.168.0.1"} +{"col1": "true", "col2": "hyunsik", "col3": 17, "col4": 59, "col5": 23, "col6": 77.9, "col7": 271.9, "col8": "hyunsik", "col9": "aHl1bnNpaw==", "col10": "192.168.0.1"} From fddeeef23188d7724c04da56614066da0139d761 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 12:05:28 +0900 Subject: [PATCH 37/55] TAJO-1344_3 --- .../tajo/plan/expr/GeneralFunctionEval.java | 19 ------ .../tajo/plan/function/FunctionInvoke.java | 8 ++- .../plan/function/FunctionInvokeContext.java | 3 + .../function/LegacyScalarFunctionInvoke.java | 8 +-- .../plan/function/PythonFunctionInvoke.java | 29 ++++------ .../function/python/PythonScriptExecutor.java | 58 ++++++++++++------- .../plan/function/stream/InputHandler.java | 38 +++++++----- .../plan/function/stream/OutputHandler.java | 15 ++--- .../stream/StreamingUDFInputHandler.java | 44 -------------- .../stream/StreamingUDFOutputHandler.java | 31 ---------- 10 files changed, 85 insertions(+), 168 deletions(-) delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 97573fa36d..fa9aca3537 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -18,7 +18,6 @@ package org.apache.tajo.plan.expr; -import com.google.common.base.Objects; import com.google.gson.annotations.Expose; import org.apache.tajo.OverridableConf; import org.apache.tajo.catalog.FunctionDesc; @@ -27,10 +26,8 @@ import org.apache.tajo.plan.function.FunctionInvoke; import org.apache.tajo.plan.function.FunctionInvokeContext; import org.apache.tajo.storage.Tuple; -import org.apache.tajo.util.TUtil; import java.io.IOException; -import java.util.Map; public class GeneralFunctionEval extends FunctionEval { protected FunctionInvoke funcInvoke; @@ -59,25 +56,9 @@ public EvalNode bind(Schema schema) { public Datum eval(Tuple tuple) { super.eval(tuple); Datum res = funcInvoke.eval(evalParams(tuple)); - funcInvoke.close(); return res; } -// @Override -// public boolean equals(Object obj) { -// if (obj instanceof GeneralFunctionEval) { -// GeneralFunctionEval other = (GeneralFunctionEval) obj; -// return super.equals(other); -// } -// -// return false; -// } - -// @Override -// public int hashCode() { -// return Objects.hashCode(funcDesc, funcInvoke); -// } - @Override public Object clone() throws CloneNotSupportedException { GeneralFunctionEval eval = (GeneralFunctionEval) super.clone(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java index c419791d30..b6d804cbe1 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java @@ -28,7 +28,11 @@ import java.io.Closeable; import java.io.IOException; -public abstract class FunctionInvoke implements Closeable { +/** + * An abstract class for actual function invocation. + * The metadata for function invocation are stored in the {@link org.apache.tajo.function.FunctionInvocation} class. + */ +public abstract class FunctionInvoke { @Expose protected FunctionDesc functionDesc; public FunctionInvoke() { @@ -62,8 +66,6 @@ public void setFunctionDesc(FunctionDesc functionDesc) throws InternalException */ public abstract Datum eval(Tuple tuple); - public abstract void close(); - @Override public boolean equals(Object o) { if (o instanceof FunctionInvoke) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java index 095ae057cc..3ddb917a52 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java @@ -26,6 +26,9 @@ import java.util.Arrays; import java.util.Map; +/** + * This class contains some metadata need to execute functions. + */ public class FunctionInvokeContext { private final OverridableConf queryContext; private final FunctionEval.ParamType[] paramTypes; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java index 1b97aaa052..e428d2c5b5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java @@ -25,6 +25,9 @@ import org.apache.tajo.storage.Tuple; import org.apache.tajo.util.TUtil; +/** + * This class invokes the legacy scala functions. + */ public class LegacyScalarFunctionInvoke extends FunctionInvoke { @Expose private GeneralFunction function; @@ -53,11 +56,6 @@ public Datum eval(Tuple tuple) { return function.eval(tuple); } - @Override - public void close() { - - } - @Override public boolean equals(Object o) { if (o instanceof LegacyScalarFunctionInvoke) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index 5c9178f19f..68dd45d0dc 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -25,6 +25,9 @@ import java.io.IOException; +/** + * This class invokes the python functions. + */ public class PythonFunctionInvoke extends FunctionInvoke implements Cloneable { private PythonScriptExecutor scriptExecutor; @@ -46,30 +49,18 @@ public void init(FunctionInvokeContext context) throws IOException { @Override public Datum eval(Tuple tuple) { + // TODO: Currently, the script executor is started and stopped for every eval() call. + // TODO: Since it internally forks a child process which executes python functions, + // TODO: frequent calls of start/stop functions will incur a large overhead. + // TODO: To avoid this problem, PythonScriptExecutor should have the same life cycle of the TaskRunner. + // TODO: In that case, we should consider the resource management problem, too. try { scriptExecutor.start(context); } catch (IOException e) { throw new RuntimeException(e); } - return scriptExecutor.eval(tuple); - } - - @Override - public void close() { + Datum res = scriptExecutor.eval(tuple); scriptExecutor.stop(); + return res; } - -// @Override -// public Object clone() throws CloneNotSupportedException { -// PythonFunctionInvoke clone = (PythonFunctionInvoke) super.clone(); -// if (context != null) { -// clone.context = context; -// } -// return clone; -// } -// -// @Override -// public int hashCode() { -// return Objects.hashCode(super.hashCode(), context); -// } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index b06e7272da..faf045915e 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -40,13 +40,21 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +/** + * {@link PythonScriptExecutor} is a script executor for python functions. + * It internally creates a child process which is responsible for executing python codes. + * Given an input tuple, it sends the tuple to the child process, and then receives a result from that. + */ public class PythonScriptExecutor { private static final Log LOG = LogFactory.getLog(PythonScriptExecutor.class); + private static final String PYTHON_LANGUAGE = "python"; private static final String PYTHON_ROOT_PATH = "/python"; - private static final String PYTHON_CONTROLLER_JAR_PATH = PYTHON_ROOT_PATH + "/controller.py"; // Relative to root of tajo jar. - private static final String PYTHON_TAJO_UTIL_PATH = PYTHON_ROOT_PATH + "/tajo_util.py"; // Relative to root of tajo jar. + private static final String TAJO_UTIL_NAME = "tajo_util.py"; + private static final String CONTROLLER_NAME = "controller.py"; + private static final String PYTHON_CONTROLLER_JAR_PATH = PYTHON_ROOT_PATH + File.separator + CONTROLLER_NAME; // Relative to root of tajo jar. + private static final String PYTHON_TAJO_UTIL_PATH = PYTHON_ROOT_PATH + File.separator + TAJO_UTIL_NAME; // Relative to root of tajo jar. private static final String DEFAULT_LOG_DIR = "/tmp/tajo-" + System.getProperty("user.name") + "/python"; // Indexes for arguments being passed to external process @@ -61,7 +69,7 @@ public class PythonScriptExecutor { private static final int CONTROLLER_LOG_FILE_PATH = 8; // Controller log file logs progress through the controller script not user code. private static final int OUT_SCHEMA = 9; // the schema of the output column - private Process process; // Handle to the externwlgns1441 + private Process process; // Handle to the external execution of python functions // all processes private ProcessErrorThread stderrThread; // thread to get process stderr private ProcessInputThread stdinThread; // thread to send input to process @@ -78,20 +86,20 @@ public class PythonScriptExecutor { private InputStream stderr; // stderr of the process private static final Object ERROR_OUTPUT = new Object(); - private static final Object NULL_OBJECT = new Object(); //BlockingQueue can't have null. Use place holder object instead. + private static final Object NULL_OBJECT = new Object(); private volatile StreamingUDFException outerrThreadsError; private FunctionInvokeContext invokeContext = null; - private FunctionSignature functionSignature; - private PythonInvocationDesc invocationDesc; - private Schema inSchema; - private Schema outSchema; - private int [] projectionCols; + private final FunctionSignature functionSignature; + private final PythonInvocationDesc invocationDesc; + private final Schema inSchema; + private final Schema outSchema; + private final int [] projectionCols; - private CSVLineSerDe lineSerDe = new CSVLineSerDe(); - private TableMeta pipeMeta; + private final CSVLineSerDe lineSerDe = new CSVLineSerDe(); + private final TableMeta pipeMeta; public PythonScriptExecutor(FunctionDesc functionDesc) { if (!functionDesc.getInvocation().hasPython()) { @@ -134,6 +142,11 @@ private StreamingCommand startUdfController() throws IOException { return sc; } + /** + * Build a command to execute external process. + * @return + * @throws IOException + */ private String[] buildCommand() throws IOException { OverridableConf queryContext = invokeContext.getQueryContext(); String[] command = new String[10]; @@ -153,7 +166,7 @@ private String[] buildCommand() throws IOException { outFileName = standardOutputRootWriteLocation + funcName + ".out"; errOutFileName = standardOutputRootWriteLocation + funcName + ".err"; - command[UDF_LANGUAGE] = "python"; + command[UDF_LANGUAGE] = PYTHON_LANGUAGE; command[PATH_TO_CONTROLLER_FILE] = getControllerPath(); int lastSeparator = filePath.lastIndexOf(File.separator) + 1; String fileName = filePath.substring(lastSeparator); @@ -161,7 +174,6 @@ private String[] buildCommand() throws IOException { command[UDF_FILE_NAME] = fileName; command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; - // TODO if (!invokeContext.getQueryContext().containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); } @@ -178,10 +190,10 @@ private String[] buildCommand() throws IOException { private void createInputHandlers() { TextLineSerializer serializer = lineSerDe.createSerializer(inSchema, pipeMeta); serializer.init(); - this.inputHandler = new StreamingUDFInputHandler(serializer); + this.inputHandler = new InputHandler(serializer); TextLineDeserializer deserializer = lineSerDe.createDeserializer(outSchema, pipeMeta, projectionCols); deserializer.init(); - this.outputHandler = new StreamingUDFOutputHandler(deserializer); + this.outputHandler = new OutputHandler(deserializer); } private void setStreams() throws IOException { @@ -227,7 +239,7 @@ private String getControllerPath() throws IOException { pythonControllerStream.close(); } controllerFile.deleteOnExit(); - File tajoUtilFile = new File(controllerFile.getParent() + "/tajo_util.py"); + File tajoUtilFile = new File(controllerFile.getParent() + File.separator + TAJO_UTIL_NAME); tajoUtilFile.deleteOnExit(); InputStream pythonUtilStream = this.getClass().getResourceAsStream(PYTHON_TAJO_UTIL_PATH); try { @@ -252,6 +264,8 @@ public Datum eval(Tuple input) { //We want it to be nothing (since that's what the user wrote). input = new VTuple(0); } + // TODO: Currently, errors occurred before executing an input are ignored. + outputQueue.clear(); inputQueue.put(input); } catch (Exception e) { throw new RuntimeException("Failed adding input to inputQueue", e); @@ -334,18 +348,18 @@ public void run() { } catch(Exception e) { if (outputQueue != null) { try { - //Give error thread a chance to check the standard error output - //for an exception message. + // Give error thread a chance to check the standard error output + // for an exception message. int attempt = 0; while (stderrThread.isAlive() && attempt < MAX_WAIT_FOR_ERROR_ATTEMPTS) { Thread.sleep(WAIT_FOR_ERROR_LENGTH); attempt++; } - //Only write this if no other error. Don't want to overwrite - //an error from the error thread. + // Only write this if no other error. Don't want to overwrite + // an error from the error thread. if (outerrThreadsError == null) { outerrThreadsError = new StreamingUDFException( - "python", "Error deserializing output. Please check that the declared outputSchema for function " + + PYTHON_LANGUAGE, "Error deserializing output. Please check that the declared outputSchema for function " + invocationDesc.getName() + " matches the data type being returned.", e); } outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. @@ -382,7 +396,7 @@ public void run() { error.append(errInput + "\n"); } } - outerrThreadsError = new StreamingUDFException("python", error.toString(), lineNumber); + outerrThreadsError = new StreamingUDFException(PYTHON_LANGUAGE, error.toString(), lineNumber); if (outputQueue != null) { outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java index e9d957ecc9..f864f4dfe7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -24,23 +24,14 @@ import java.io.OutputStream; /** - * {@link InputHandler} is responsible for handling the input to the - * Tajo-Streaming external command. + * {@link InputHandler} is responsible for handling the input to the Tajo-Streaming external command. * - * The managed executable could be fed input in a {@link InputType#SYNCHRONOUS} - * manner via its stdin or in an {@link InputType#ASYNCHRONOUS} - * manner via an external file which is subsequently read by the executable. */ -public abstract class InputHandler { +public class InputHandler { private final static byte[] END_OF_RECORD_DELIM = "|_\n".getBytes(); private final static byte[] END_OF_STREAM = ("C" + "\\x04" + "|_\n").getBytes(); - /** - * - */ - public enum InputType {SYNCHRONOUS, ASYNCHRONOUS} - /** * The serializer to be used to send data to the managed process. * @@ -54,6 +45,10 @@ public enum InputType {SYNCHRONOUS, ASYNCHRONOUS} // flag to mark if close() has already been called protected boolean alreadyClosed = false; + public InputHandler(TextLineSerializer serializer) { + this.serializer = serializer; + } + /** * Send the given input Tuple to the managed executable. * @@ -78,11 +73,22 @@ public void putNext(Tuple t) throws IOException { * @throws IOException */ public synchronized void close(Process process) throws IOException { - if(!alreadyClosed) { - alreadyClosed = true; - out.flush(); - out.close(); - out = null; + try { + if (!alreadyClosed) { + alreadyClosed = true; + out.flush(); + out.close(); + out = null; + } + } catch(IOException e) { + // check if we got an exception because + // the process actually completed and we were + // trying to flush and close it's stdin + if (process == null || process.exitValue() != 0) { + // the process had not terminated normally + // throw the exception we got + throw e; + } } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java index 3003de3fd9..fcb3029a76 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java @@ -34,18 +34,11 @@ /** * {@link OutputHandler} is responsible for handling the output of the * Tajo-Streaming external command. - * - * The output of the managed executable could be fetched in a - * {@link OutputType#SYNCHRONOUS} manner via its stdout or in an - * {@link OutputType#ASYNCHRONOUS} manner via an external file to which the - * process wrote its output. */ -public abstract class OutputHandler { +public class OutputHandler { private static int DEFAULT_BUFFER = 64 * 1024; public static final Object END_OF_OUTPUT = new Object(); - private static final byte[] DEFAULT_RECORD_DELIM = new byte[] {'\n'}; - - public enum OutputType {SYNCHRONOUS, ASYNCHRONOUS} + private static final byte[] DEFAULT_RECORD_DELIM = ",".getBytes(); protected TextLineDeserializer deserializer; @@ -66,6 +59,10 @@ public enum OutputType {SYNCHRONOUS, ASYNCHRONOUS} // flag to mark if close() has already been called protected boolean alreadyClosed = false; + public OutputHandler(TextLineDeserializer deserializer) { + this.deserializer = deserializer; + } + /** * Bind the OutputHandler to the InputStream * from which to read the output data of the managed process. diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java deleted file mode 100644 index e7354f8672..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFInputHandler.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Lisensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.stream; - -import java.io.IOException; - -public class StreamingUDFInputHandler extends InputHandler { - - public StreamingUDFInputHandler(TextLineSerializer serializer) { - this.serializer = serializer; - } - - @Override - public synchronized void close(Process process) throws IOException { - try { - super.close(process); - } catch(IOException e) { - // check if we got an exception because - // the process actually completed and we were - // trying to flush and close it's stdin - if (process == null || process.exitValue() != 0) { - // the process had not terminated normally - // throw the exception we got - throw e; - } - } - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java deleted file mode 100644 index 5b8e2ecaff..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFOutputHandler.java +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Lisensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.stream; - -public class StreamingUDFOutputHandler extends OutputHandler { - - public StreamingUDFOutputHandler(TextLineDeserializer deserializer) { - this.deserializer = deserializer; - } - - @Override - protected byte[] getRecordDelimiter() { - return ",".getBytes(); - } -} From 4b645f4c953a8d330925d72a729f664164666411 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 12:15:50 +0900 Subject: [PATCH 38/55] TAJO-1344_3 --- .../function/python/PythonScriptEngine.java | 11 ----- .../function/python/PythonScriptExecutor.java | 14 +++---- .../function/python/TajoScriptEngine.java | 41 ------------------- .../TextFieldSerializerDeserializer.java | 1 - 4 files changed, 7 insertions(+), 60 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index e4ade18cd4..d23837ac7d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -34,7 +34,6 @@ import java.net.URI; import java.nio.charset.Charset; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.regex.Pattern; @@ -75,16 +74,6 @@ private static TajoDataTypes.DataType[] createParamTypes(int paramNum) { return paramTypes; } - @Override - protected String getScriptingLang() { - return "streaming_python"; - } - - @Override - protected Map getParamsFromVariables() throws IOException { - throw new IOException("Unsupported Operation"); - } - private static final Pattern pSchema = Pattern.compile("^\\s*\\W+outputType.*"); private static final Pattern pDef = Pattern.compile("^\\s*def\\s+(\\w+)\\s*.+"); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index faf045915e..52df0183f4 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -259,9 +259,9 @@ public Datum eval(Tuple input) { try { if (input == null) { - //When nothing is passed into the UDF the tuple - //being sent is the full tuple for the relation. - //We want it to be nothing (since that's what the user wrote). + // When nothing is passed into the UDF the tuple + // being sent is the full tuple for the relation. + // We want it to be nothing (since that's what the user wrote). input = new VTuple(0); } // TODO: Currently, errors occurred before executing an input are ignored. @@ -362,7 +362,7 @@ public void run() { PYTHON_LANGUAGE, "Error deserializing output. Please check that the declared outputSchema for function " + invocationDesc.getName() + " matches the data type being returned.", e); } - outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. + outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. } catch(InterruptedException ie) { LOG.error(ie); } @@ -384,8 +384,8 @@ public void run() { BufferedReader reader = new BufferedReader( new InputStreamReader(stderr, Charsets.UTF_8)); while ((errInput = reader.readLine()) != null) { - //First line of error stream is usually the line number of error. - //If its not a number just treat it as first line of error message. + // First line of error stream is usually the line number of error. + // If its not a number just treat it as first line of error message. if (lineNumber == null) { try { lineNumber = Integer.valueOf(errInput); @@ -398,7 +398,7 @@ public void run() { } outerrThreadsError = new StreamingUDFException(PYTHON_LANGUAGE, error.toString(), lineNumber); if (outputQueue != null) { - outputQueue.put(ERROR_OUTPUT); //Need to wake main thread. + outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. } if (stderr != null) { stderr.close(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 76d3e6a602..b666262771 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -22,14 +22,10 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.InputStream; -import java.io.IOException; import java.net.URI; -import java.util.Map; public abstract class TajoScriptEngine { - public static final String NAMESPACE_SEPARATOR = "."; - /** * Open a stream load a script locally or in the classpath * @param scriptPath the path of the script @@ -45,25 +41,6 @@ protected static InputStream getScriptAsStream(URI scriptPath) { } catch (FileNotFoundException e) { throw new IllegalStateException("could not find existing file "+scriptPath, e); } -// } else { -// // Try system, current and context classloader. -// is = TajoScriptEngine.class.getResourceAsStream(scriptPath); -// if (is == null) { -// is = getResourceUsingClassLoader(scriptPath, TajoScriptEngine.class.getClassLoader()); -// } -// if (is == null) { -// is = getResourceUsingClassLoader(scriptPath, Thread.currentThread().getContextClassLoader()); -// } -// if (is == null && !file.isAbsolute()) { -// String path = "/" + scriptPath; -// is = TajoScriptEngine.class.getResourceAsStream(path); -// if (is == null) { -// is = getResourceUsingClassLoader(path, TajoScriptEngine.class.getClassLoader()); -// } -// if (is == null) { -// is = getResourceUsingClassLoader(path, Thread.currentThread().getContextClassLoader()); -// } -// } } if (is == null) { @@ -72,22 +49,4 @@ protected static InputStream getScriptAsStream(URI scriptPath) { } return is; } - - private static InputStream getResourceUsingClassLoader(String fullFilename, ClassLoader loader) { - if (loader != null) { - return loader.getResourceAsStream(fullFilename); - } - return null; - } - - /** - * Gets ScriptEngine classname or keyword for the scripting language - */ - protected abstract String getScriptingLang(); - - /** - * Returns a map from local variable names to their values - * @throws java.io.IOException - */ - protected abstract Map getParamsFromVariables() throws IOException; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java index 8d7dff5b1b..cd9518b127 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/TextFieldSerializerDeserializer.java @@ -245,7 +245,6 @@ public Datum deserialize(ByteBuf buf, Column col, int columnIndex, ByteBuf nullC byte[] bytes = new byte[buf.readableBytes()]; buf.readBytes(bytes); datum = DatumFactory.createBlob(Base64.decodeBase64(bytes)); -// datum = DatumFactory.createBlob(bytes); break; } default: From ea0f548025cca76ee99631b5ebc78422c85340a0 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 13:54:28 +0900 Subject: [PATCH 39/55] TAJO-1344_3 --- tajo-core/src/main/resources/python/controller.py | 13 +++++++++---- .../plan/function/python/PythonScriptExecutor.java | 10 ++++++---- .../tajo/plan/function/python/TajoScriptEngine.java | 7 ++++++- 3 files changed, 21 insertions(+), 9 deletions(-) diff --git a/tajo-core/src/main/resources/python/controller.py b/tajo-core/src/main/resources/python/controller.py index 7be02c458e..3098146cd6 100644 --- a/tajo-core/src/main/resources/python/controller.py +++ b/tajo-core/src/main/resources/python/controller.py @@ -95,9 +95,9 @@ def main(self, sys.path.append(cache_path) sys.path.append('.') - # logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) - # logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the " - # "INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") + logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) + logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the " + "INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") input_str = self.get_next_input() @@ -106,6 +106,7 @@ def main(self, except: # These errors should always be caused by user code. write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) + logging.info('write_user_exception1') self.close_controller(-1) # if udf_logging.udf_log_level != logging.DEBUG: @@ -114,7 +115,7 @@ def main(self, # else: # sys.stdout = self.log_stream - should_log = False + should_log = True log_message = logging.info if udf_logging.udf_log_level == logging.DEBUG: should_log = True @@ -131,6 +132,7 @@ def main(self, except: # Capture errors where the user passes in bad data. write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) + logging.info('write_user_exception2') self.close_controller(-3) try: @@ -140,6 +142,7 @@ def main(self, except: # These errors should always be caused by user code. write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) + logging.info('write_user_exception3') self.close_controller(-2) output = serialize_output(func_output, output_schema) @@ -152,6 +155,7 @@ def main(self, # and pig- not with user code. import traceback traceback.print_exc(file=self.stream_error) + logging.info('traceback') sys.exit(-3) sys.stdout.flush() @@ -190,6 +194,7 @@ def get_next_input(self): def close_controller(self, exit_code): sys.stderr.close() self.stream_error.write("\n") + logging.info('last') self.stream_error.close() sys.stdout.close() self.stream_output.write("\n") diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index 52df0183f4..73087e626d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -157,6 +157,7 @@ private String[] buildCommand() throws IOException { LOG.warn("Currently, logging is not supported for the python controller."); standardOutputRootWriteLocation = invokeContext.getQueryContext().get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); } + standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; String funcName = invocationDesc.getName(); @@ -264,8 +265,7 @@ public Datum eval(Tuple input) { // We want it to be nothing (since that's what the user wrote). input = new VTuple(0); } - // TODO: Currently, errors occurred before executing an input are ignored. - outputQueue.clear(); + inputQueue.put(input); } catch (Exception e) { throw new RuntimeException("Failed adding input to inputQueue", e); @@ -362,7 +362,8 @@ public void run() { PYTHON_LANGUAGE, "Error deserializing output. Please check that the declared outputSchema for function " + invocationDesc.getName() + " matches the data type being returned.", e); } - outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. + // TODO: Currently, errors occurred before executing an input are ignored. +// outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. } catch(InterruptedException ie) { LOG.error(ie); } @@ -398,7 +399,8 @@ public void run() { } outerrThreadsError = new StreamingUDFException(PYTHON_LANGUAGE, error.toString(), lineNumber); if (outputQueue != null) { - outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. + // TODO: Currently, errors occurred before executing an input are ignored. +// outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. } if (stderr != null) { stderr.close(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index b666262771..b915318480 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -34,7 +34,12 @@ public abstract class TajoScriptEngine { */ protected static InputStream getScriptAsStream(URI scriptPath) { InputStream is = null; - File file = new File(scriptPath); + File file; + try { + file = new File(scriptPath); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("path: " + scriptPath, e); + } if (file.exists()) { try { is = new FileInputStream(file); From 02a06a2f59529886dee776ec1be98a20fa4d07f1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 17:05:22 +0900 Subject: [PATCH 40/55] Fix thread cleanup --- .../src/main/resources/python/controller.py | 10 ++-- .../plan/function/PythonFunctionInvoke.java | 12 +++- .../function/python/PythonScriptExecutor.java | 58 ++++++++++++++----- .../function/stream/CSVLineSerializer.java | 1 + .../plan/serder/EvalNodeDeserializer.java | 1 - 5 files changed, 57 insertions(+), 25 deletions(-) diff --git a/tajo-core/src/main/resources/python/controller.py b/tajo-core/src/main/resources/python/controller.py index 3098146cd6..077864d455 100644 --- a/tajo-core/src/main/resources/python/controller.py +++ b/tajo-core/src/main/resources/python/controller.py @@ -105,8 +105,8 @@ def main(self, func = __import__(module_name, globals(), locals(), [func_name], -1).__dict__[func_name] except: # These errors should always be caused by user code. - write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) logging.info('write_user_exception1') + write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-1) # if udf_logging.udf_log_level != logging.DEBUG: @@ -131,8 +131,8 @@ def main(self, log_message("Deserialized Input: %s" % (unicode(inputs))) except: # Capture errors where the user passes in bad data. - write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) logging.info('write_user_exception2') + write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-3) try: @@ -141,8 +141,8 @@ def main(self, log_message("UDF Output: %s" % (unicode(func_output))) except: # These errors should always be caused by user code. - write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) logging.info('write_user_exception3') + write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-2) output = serialize_output(func_output, output_schema) @@ -154,8 +154,8 @@ def main(self, # This should only catch internal exceptions with the controller # and pig- not with user code. import traceback - traceback.print_exc(file=self.stream_error) logging.info('traceback') + traceback.print_exc(file=self.stream_error) sys.exit(-3) sys.stdout.flush() @@ -193,8 +193,8 @@ def get_next_input(self): def close_controller(self, exit_code): sys.stderr.close() - self.stream_error.write("\n") logging.info('last') + self.stream_error.write("\n") self.stream_error.close() sys.stdout.close() self.stream_output.write("\n") diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index 68dd45d0dc..903a24793c 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -52,15 +52,21 @@ public Datum eval(Tuple tuple) { // TODO: Currently, the script executor is started and stopped for every eval() call. // TODO: Since it internally forks a child process which executes python functions, // TODO: frequent calls of start/stop functions will incur a large overhead. - // TODO: To avoid this problem, PythonScriptExecutor should have the same life cycle of the TaskRunner. - // TODO: In that case, we should consider the resource management problem, too. + // TODO: To avoid this problem, PythonScriptExecutor should have the same life cycle with the TaskRunner. + // TODO: In addition, we should consider the resource management problem, too. try { scriptExecutor.start(context); } catch (IOException e) { throw new RuntimeException(e); } Datum res = scriptExecutor.eval(tuple); - scriptExecutor.stop(); + try { + scriptExecutor.stop(); + } catch (IOException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } return res; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index 73087e626d..6406ed0d8d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -39,6 +39,7 @@ import java.io.*; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; /** * {@link PythonScriptExecutor} is a script executor for python functions. @@ -101,6 +102,8 @@ public class PythonScriptExecutor { private final CSVLineSerDe lineSerDe = new CSVLineSerDe(); private final TableMeta pipeMeta; + private boolean isStopped = false; + public PythonScriptExecutor(FunctionDesc functionDesc) { if (!functionDesc.getInvocation().hasPython()) { throw new IllegalStateException("Function type must be python"); @@ -121,6 +124,7 @@ public PythonScriptExecutor(FunctionDesc functionDesc) { } public void start(FunctionInvokeContext context) throws IOException { + isStopped = false; this.invokeContext = context; this.inputQueue = new ArrayBlockingQueue(1); this.outputQueue = new ArrayBlockingQueue(2); @@ -130,7 +134,22 @@ public void start(FunctionInvokeContext context) throws IOException { startThreads(); } - public void stop() { + public void stop() throws IOException, InterruptedException { + isStopped = true; + if (stdin != null) { + stdin.close(); + } + if (stdout != null) { + stdout.close(); + } + if (stderr != null) { + stderr.close(); + } + inputHandler.close(process); + outputHandler.close(); + stdinThread.join(); + stderrThread.join(); + stdoutThread.join(); process.destroy(); } @@ -157,7 +176,8 @@ private String[] buildCommand() throws IOException { LOG.warn("Currently, logging is not supported for the python controller."); standardOutputRootWriteLocation = invokeContext.getQueryContext().get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); } - standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; +// standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; + standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; String funcName = invocationDesc.getName(); @@ -303,15 +323,19 @@ class ProcessInputThread extends Thread { public void run() { try { - while (true) { - Tuple inputTuple = inputQueue.take(); - inputHandler.putNext(inputTuple); + while (!isStopped) { + Tuple inputTuple = inputQueue.poll(10, TimeUnit.MILLISECONDS); + if (inputTuple != null) { + inputHandler.putNext(inputTuple); + } try { stdin.flush(); } catch(Exception e) { return; } } + } catch (InterruptedException e) { + } catch (Exception e) { LOG.error(e); } @@ -334,7 +358,7 @@ public void run() { try { o = outputHandler.getNext().get(0); - while (o != OutputHandler.END_OF_OUTPUT) { + while (!isStopped && o != OutputHandler.END_OF_OUTPUT) { if (o != null) { outputQueue.put(o); } @@ -363,7 +387,7 @@ public void run() { invocationDesc.getName() + " matches the data type being returned.", e); } // TODO: Currently, errors occurred before executing an input are ignored. -// outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. + outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. } catch(InterruptedException ie) { LOG.error(ie); } @@ -384,7 +408,7 @@ public void run() { String errInput; BufferedReader reader = new BufferedReader( new InputStreamReader(stderr, Charsets.UTF_8)); - while ((errInput = reader.readLine()) != null) { + while (!isStopped && ((errInput = reader.readLine()) != null)) { // First line of error stream is usually the line number of error. // If its not a number just treat it as first line of error message. if (lineNumber == null) { @@ -397,14 +421,16 @@ public void run() { error.append(errInput + "\n"); } } - outerrThreadsError = new StreamingUDFException(PYTHON_LANGUAGE, error.toString(), lineNumber); - if (outputQueue != null) { - // TODO: Currently, errors occurred before executing an input are ignored. -// outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. - } - if (stderr != null) { - stderr.close(); - stderr = null; + if (!isStopped) { + outerrThreadsError = new StreamingUDFException(PYTHON_LANGUAGE, error.toString(), lineNumber); + if (outputQueue != null) { + // TODO: Currently, errors occurred before executing an input are ignored. + outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. + } + if (stderr != null) { + stderr.close(); + stderr = null; + } } } catch (IOException e) { LOG.info("Process Ended", e); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java index 10ae9eab47..e1c7375f50 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/CSVLineSerializer.java @@ -79,6 +79,7 @@ public int serialize(OutputStream out, Tuple input) throws IOException { @Override public void release() { + } private static String getTypeString(Datum val) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 355c695e5a..0a60f80ea6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -35,7 +35,6 @@ import org.apache.tajo.exception.InternalException; import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.function.AggFunction; -import org.apache.tajo.plan.function.GeneralFunction; import org.apache.tajo.plan.logical.WindowSpec; import org.apache.tajo.plan.serder.PlanProto.WinFunctionEvalSpec; From 74415f1b518dd6f44294f59ed96111d58d94dbc9 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 17:58:19 +0900 Subject: [PATCH 41/55] Remove thread communication --- .../java/org/apache/tajo/worker/Task.java | 3 +++ .../function/python/PythonScriptExecutor.java | 27 ++++++++++--------- 2 files changed, 18 insertions(+), 12 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index bec22b962e..c273fe9cbf 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -40,6 +40,7 @@ import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.PythonScriptExecutor; import org.apache.tajo.plan.serder.LogicalNodeDeserializer; import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.engine.planner.physical.PhysicalExec; @@ -102,6 +103,8 @@ public class Task { private Schema finalSchema = null; private TupleComparator sortComp = null; + private PythonScriptExecutor pythonExecutor; + public Task(String taskRunnerId, Path baseDir, TaskAttemptId taskId, diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index 6406ed0d8d..03d792392f 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -136,6 +136,7 @@ public void start(FunctionInvokeContext context) throws IOException { public void stop() throws IOException, InterruptedException { isStopped = true; + process.destroy(); if (stdin != null) { stdin.close(); } @@ -147,10 +148,9 @@ public void stop() throws IOException, InterruptedException { } inputHandler.close(process); outputHandler.close(); - stdinThread.join(); - stderrThread.join(); - stdoutThread.join(); - process.destroy(); +// stdinThread.join(); +// stderrThread.join(); +// stdoutThread.join(); } private StreamingCommand startUdfController() throws IOException { @@ -228,14 +228,14 @@ private void setStreams() throws IOException { } private void startThreads() { - stdinThread = new ProcessInputThread(); - stdinThread.start(); +// stdinThread = new ProcessInputThread(); +// stdinThread.start(); - stdoutThread = new ProcessOutputThread(); - stdoutThread.start(); +// stdoutThread = new ProcessOutputThread(); +// stdoutThread.start(); - stderrThread = new ProcessErrorThread(); - stderrThread.start(); +// stderrThread = new ProcessErrorThread(); +// stderrThread.start(); } /** @@ -286,14 +286,17 @@ public Datum eval(Tuple input) { input = new VTuple(0); } - inputQueue.put(input); +// inputQueue.put(input); + inputHandler.putNext(input); + stdin.flush(); } catch (Exception e) { throw new RuntimeException("Failed adding input to inputQueue", e); } Object o = null; try { if (outputQueue != null) { - o = outputQueue.take(); +// o = outputQueue.take(); + o = outputHandler.getNext().get(0); if (o == NULL_OBJECT) { o = null; } From a3bc8cf89932d779e1c5e103b35344a6545d8d00 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Wed, 8 Apr 2015 19:39:16 +0900 Subject: [PATCH 42/55] TAJO-1344_3 --- .../plan/function/PythonFunctionInvoke.java | 27 ++++++++++--------- .../function/python/PythonScriptExecutor.java | 20 +++++++++++++- .../plan/function/stream/OutputHandler.java | 1 + 3 files changed, 35 insertions(+), 13 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index 903a24793c..10ce5ec3ff 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -43,7 +43,7 @@ public PythonFunctionInvoke(FunctionDesc functionDesc) { @Override public void init(FunctionInvokeContext context) throws IOException { - this.scriptExecutor = new PythonScriptExecutor(functionDesc); +// this.scriptExecutor = new PythonScriptExecutor(functionDesc); this.context = context; } @@ -54,19 +54,22 @@ public Datum eval(Tuple tuple) { // TODO: frequent calls of start/stop functions will incur a large overhead. // TODO: To avoid this problem, PythonScriptExecutor should have the same life cycle with the TaskRunner. // TODO: In addition, we should consider the resource management problem, too. - try { - scriptExecutor.start(context); - } catch (IOException e) { - throw new RuntimeException(e); + if (scriptExecutor == null) { + this.scriptExecutor = new PythonScriptExecutor(functionDesc); + try { + scriptExecutor.start(context); + } catch (IOException e) { + throw new RuntimeException(e); + } } Datum res = scriptExecutor.eval(tuple); - try { - scriptExecutor.stop(); - } catch (IOException e) { - throw new RuntimeException(e); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } +// try { +// scriptExecutor.stop(); +// } catch (IOException e) { +// throw new RuntimeException(e); +// } catch (InterruptedException e) { +// throw new RuntimeException(e); +// } return res; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index 03d792392f..f8f80da8d7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -134,7 +134,7 @@ public void start(FunctionInvokeContext context) throws IOException { startThreads(); } - public void stop() throws IOException, InterruptedException { + public void shutdown() throws IOException, InterruptedException { isStopped = true; process.destroy(); if (stdin != null) { @@ -148,6 +148,7 @@ public void stop() throws IOException, InterruptedException { } inputHandler.close(process); outputHandler.close(); + LOG.info("shutdowned"); // stdinThread.join(); // stderrThread.join(); // stdoutThread.join(); @@ -158,6 +159,8 @@ private StreamingCommand startUdfController() throws IOException { ProcessBuilder processBuilder = StreamingUtil.createProcess(invokeContext.getQueryContext(), sc); process = processBuilder.start(); + Runtime.getRuntime().addShutdownHook(new ProcessKiller()); + return sc; } @@ -442,4 +445,19 @@ public void run() { } } } + + class ProcessKiller extends Thread { + public ProcessKiller() { + setDaemon(true); + } + public void run() { + try { + shutdown(); + } catch (IOException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java index fcb3029a76..eeec77b20f 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java @@ -91,6 +91,7 @@ public Tuple getNext() throws IOException { if (!readValue()) { return null; } + buf.clear(); buf.writeBytes(currValue.getBytes()); try { deserializer.deserialize(buf, tuple); From 054200d031ac8142054aac0d3dc743e143a0a9ed Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 10 Apr 2015 16:35:10 +0900 Subject: [PATCH 43/55] TAJO-1344_3 --- .../apache/tajo/engine/planner/Projector.java | 2 +- .../rules/GlobalPlanEqualityTester.java | 2 +- .../planner/physical/AggregationExec.java | 2 +- .../planner/physical/BSTIndexScanExec.java | 2 +- .../planner/physical/CommonJoinExec.java | 2 +- .../DistinctGroupbyFirstAggregationExec.java | 2 +- .../DistinctGroupbyHashAggregationExec.java | 2 +- .../DistinctGroupbySecondAggregationExec.java | 2 +- .../DistinctGroupbyThirdAggregationExec.java | 2 +- .../engine/planner/physical/EvalExprExec.java | 2 +- .../physical/HashLeftOuterJoinExec.java | 4 +- .../engine/planner/physical/HavingExec.java | 2 +- .../planner/physical/SelectionExec.java | 2 +- .../engine/planner/physical/SeqScanExec.java | 4 +- .../planner/physical/WindowAggExec.java | 2 +- .../tajo/master/exec/QueryExecutor.java | 85 +++++++---- .../java/org/apache/tajo/worker/Task.java | 55 +++++-- .../tajo/worker/TaskAttemptContext.java | 7 + .../apache/tajo/engine/eval/ExprTestBase.java | 17 ++- .../apache/tajo/engine/eval/TestEvalTree.java | 68 ++++----- .../tajo/engine/eval/TestEvalTreeUtil.java | 20 +-- .../org/apache/tajo/plan/ExprAnnotator.java | 3 +- .../org/apache/tajo/plan/LogicalPlanner.java | 20 ++- .../apache/tajo/plan/expr/AlgebraicUtil.java | 6 +- .../tajo/plan/expr/BetweenPredicateEval.java | 28 ++-- .../apache/tajo/plan/expr/EvalContext.java | 45 ++++++ .../org/apache/tajo/plan/expr/EvalNode.java | 5 +- .../apache/tajo/plan/expr/EvalTreeUtil.java | 4 +- .../org/apache/tajo/plan/expr/FieldEval.java | 4 +- .../apache/tajo/plan/expr/FunctionEval.java | 4 +- .../tajo/plan/expr/GeneralFunctionEval.java | 7 +- .../plan/expr/PatternMatchPredicateEval.java | 4 +- .../exprrewrite/EvalTreeOptimizationRule.java | 3 +- .../plan/exprrewrite/EvalTreeOptimizer.java | 1 + .../exprrewrite/rules/ConstantFolding.java | 6 +- .../tajo/plan/function/FunctionInvoke.java | 1 + .../plan/function/FunctionInvokeContext.java | 17 ++- .../function/LegacyScalarFunctionInvoke.java | 1 + .../plan/function/PythonFunctionInvoke.java | 19 ++- .../function/python/PythonScriptExecutor.java | 27 ++-- .../plan/function/python/ScriptExecutor.java | 14 ++ .../rules/LogicalPlanEqualityTester.java | 2 +- .../rules/PartitionedTableRewriter.java | 2 +- .../plan/serder/EvalNodeDeserializer.java | 8 +- .../plan/serder/LogicalNodeDeserializer.java | 137 ++++++++++-------- 45 files changed, 419 insertions(+), 235 deletions(-) create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java create mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java index cec18620d8..a73478f6d8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/Projector.java @@ -63,7 +63,7 @@ public Projector(TaskAttemptContext context, Schema inSchema, Schema outSchema, public void init() { for (EvalNode eval : evals) { - eval.bind(inSchema); + eval.bind(context.getEvalContext(), inSchema); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java index e2fd47f944..e55a258b4e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java @@ -51,7 +51,7 @@ public MasterPlan rewrite(MasterPlan plan) { LogicalNode node = eb.getPlan(); if (node != null) { PlanProto.LogicalNodeTree tree = LogicalNodeSerializer.serialize(node); - LogicalNode deserialize = LogicalNodeDeserializer.deserialize(plan.getContext(), tree); + LogicalNode deserialize = LogicalNodeDeserializer.deserialize(plan.getContext(), null, tree); assert node.deepEquals(deserialize); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java index 6d9e38a796..4b53b39501 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/AggregationExec.java @@ -63,7 +63,7 @@ public AggregationExec(final TaskAttemptContext context, GroupbyNode plan, public void init() throws IOException { super.init(); for (EvalNode aggFunction : aggFunctions) { - aggFunction.bind(inSchema); + aggFunction.bind(context.getEvalContext(), inSchema); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java index be6c046356..806d34c0f1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/BSTIndexScanExec.java @@ -72,7 +72,7 @@ public void init() throws IOException { super.init(); progress = 0.0f; if (qual != null) { - qual.bind(inSchema); + qual.bind(context.getEvalContext(), inSchema); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java index 0781041998..2535edffdc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/CommonJoinExec.java @@ -54,7 +54,7 @@ public CommonJoinExec(TaskAttemptContext context, JoinNode plan, PhysicalExec ou public void init() throws IOException { super.init(); if (hasJoinQual) { - joinQual.bind(inSchema); + joinQual.bind(context.getEvalContext(), inSchema); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java index 37bc5a7523..94429a0d22 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyFirstAggregationExec.java @@ -250,7 +250,7 @@ private NonDistinctHashAggregator(GroupbyNode groupbyNode) throws IOException { } for (AggregationFunctionCallEval eachFunction: aggFunctions) { - eachFunction.bind(inSchema); + eachFunction.bind(context.getEvalContext(), inSchema); eachFunction.setFirstPhase(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java index e96e750b61..0f25d6cbb9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyHashAggregationExec.java @@ -375,7 +375,7 @@ public HashAggregator(GroupbyNode groupbyNode, Schema schema) throws IOException } for (EvalNode aggFunction : aggFunctions) { - aggFunction.bind(schema); + aggFunction.bind(context.getEvalContext(), schema); } tupleSize = groupingKeyIds.length + aggFunctionsNum; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java index 7b01a9b277..b39439038f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbySecondAggregationExec.java @@ -119,7 +119,7 @@ public void init() throws IOException { nonDistinctAggrFunctions = eachGroupby.getAggFunctions(); if (nonDistinctAggrFunctions != null) { for (AggregationFunctionCallEval eachFunction: nonDistinctAggrFunctions) { - eachFunction.bind(inSchema); + eachFunction.bind(context.getEvalContext(), inSchema); eachFunction.setIntermediatePhase(); } nonDistinctAggrContexts = new FunctionContext[nonDistinctAggrFunctions.length]; diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java index 7bd71e25ec..e71976c9ee 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/DistinctGroupbyThirdAggregationExec.java @@ -252,7 +252,7 @@ public DistinctFinalAggregator(int seq, int inTupleIndex, int outTupleIndex, Gro aggrFunctions = groupbyNode.getAggFunctions(); if (aggrFunctions != null) { for (AggregationFunctionCallEval eachFunction: aggrFunctions) { - eachFunction.bind(inSchema); + eachFunction.bind(context.getEvalContext(), inSchema); eachFunction.setFinalPhase(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java index 32ec772248..4581b4a2fc 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/EvalExprExec.java @@ -41,7 +41,7 @@ public void init() throws IOException { super.init(); progress = 0.0f; for (Target target : plan.getTargets()) { - target.getEvalTree().bind(inSchema); + target.getEvalTree().bind(context.getEvalContext(), inSchema); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java index fa9ba940ba..6f573d0e06 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HashLeftOuterJoinExec.java @@ -121,9 +121,9 @@ public HashLeftOuterJoinExec(TaskAttemptContext context, JoinNode plan, Physical rightNumCols = rightChild.getSchema().size(); - joinQual.bind(inSchema); + joinQual.bind(context.getEvalContext(), inSchema); if (joinFilter != null) { - joinFilter.bind(inSchema); + joinFilter.bind(context.getEvalContext(), inSchema); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java index b71c770c9e..6897e92dd9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/HavingExec.java @@ -39,7 +39,7 @@ public HavingExec(TaskAttemptContext context, @Override public void init() throws IOException { super.init(); - qual.bind(inSchema); + qual.bind(context.getEvalContext(), inSchema); } @Override diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java index c090fa75f0..7f5bbe96d9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SelectionExec.java @@ -39,7 +39,7 @@ public SelectionExec(TaskAttemptContext context, @Override public void init() throws IOException { super.init(); - qual.bind(inSchema); + qual.bind(context.getEvalContext(), inSchema); } @Override diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java index 671555c791..ff9477fa78 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/SeqScanExec.java @@ -109,7 +109,7 @@ private void rewriteColumnPartitionedTableSchema() throws IOException { FieldEval targetExpr = new FieldEval(column); Datum datum = NullDatum.get(); if (partitionRow != null) { - targetExpr.bind(columnPartitionSchema); + targetExpr.bind(context.getEvalContext(), columnPartitionSchema); datum = targetExpr.eval(partitionRow); } ConstEval constExpr = new ConstEval(datum); @@ -163,7 +163,7 @@ public void init() throws IOException { super.init(); if (plan.hasQual()) { - qual.bind(inSchema); + qual.bind(context.getEvalContext(), inSchema); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/WindowAggExec.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/WindowAggExec.java index 2f1fc467f0..05b0418113 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/WindowAggExec.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/physical/WindowAggExec.java @@ -183,7 +183,7 @@ public WindowAggExec(TaskAttemptContext context, WindowAggNode plan, PhysicalExe public void init() throws IOException { super.init(); for (EvalNode functionEval : functions) { - functionEval.bind(inSchema); + functionEval.bind(context.getEvalContext(), inSchema); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index 2eb3c5f752..8165d03e8d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -18,6 +18,7 @@ package org.apache.tajo.master.exec; +import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -47,6 +48,10 @@ import org.apache.tajo.master.exec.prehook.CreateTableHook; import org.apache.tajo.master.exec.prehook.DistributedQueryHookManager; import org.apache.tajo.master.exec.prehook.InsertIntoHook; +import org.apache.tajo.plan.expr.EvalContext; +import org.apache.tajo.plan.expr.GeneralFunctionEval; +import org.apache.tajo.plan.function.python.PythonScriptExecutor; +import org.apache.tajo.plan.function.python.ScriptExecutor; import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule; import org.apache.tajo.querymaster.*; import org.apache.tajo.session.Session; @@ -84,7 +89,7 @@ public QueryExecutor(TajoMaster.MasterContext context, DDLExecutor ddlExecutor) } public SubmitQueryResponse execute(QueryContext queryContext, Session session, String sql, String jsonExpr, - LogicalPlan plan) throws Exception { + LogicalPlan plan, EvalContext evalContext) throws Exception { SubmitQueryResponse.Builder response = SubmitQueryResponse.newBuilder(); response.setIsForwarded(false); @@ -113,11 +118,9 @@ public SubmitQueryResponse execute(QueryContext queryContext, Session session, S } else if (PlannerUtil.checkIfSimpleQuery(plan)) { execSimpleQuery(queryContext, session, sql, plan, response); - // NonFromQuery indicates a form of 'select a, x+y;' } else if (PlannerUtil.checkIfNonFromQuery(plan)) { - execNonFromQuery(queryContext, session, sql, plan, response); - + execNonFromQuery(queryContext, plan, response, evalContext); } else { // it requires distributed execution. So, the query is forwarded to a query master. executeDistributedQuery(queryContext, session, plan, sql, jsonExpr, response); @@ -263,37 +266,67 @@ public void execSimpleQuery(QueryContext queryContext, Session session, String q response.setResultCode(ClientProtos.ResultCode.OK); } - public void execNonFromQuery(QueryContext queryContext, Session session, String query, - LogicalPlan plan, SubmitQueryResponse.Builder responseBuilder) throws Exception { + public void execNonFromQuery(QueryContext queryContext, LogicalPlan plan, SubmitQueryResponse.Builder responseBuilder, + EvalContext evalContext) throws Exception { LogicalRootNode rootNode = plan.getRootBlock().getRoot(); Target[] targets = plan.getRootBlock().getRawTargets(); if (targets == null) { throw new PlanningException("No targets"); } - final Tuple outTuple = new VTuple(targets.length); + try { + // start script executor + startScriptExecutors(queryContext, evalContext, targets); + final Tuple outTuple = new VTuple(targets.length); + for (int i = 0; i < targets.length; i++) { + EvalNode eval = targets[i].getEvalTree(); + eval.bind(evalContext, null); + outTuple.put(i, eval.eval(null)); + } + boolean isInsert = rootNode.getChild() != null && rootNode.getChild().getType() == NodeType.INSERT; + if (isInsert) { + InsertNode insertNode = rootNode.getChild(); + insertNonFromQuery(queryContext, insertNode, responseBuilder); + } else { + Schema schema = PlannerUtil.targetToSchema(targets); + RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(schema); + byte[] serializedBytes = encoder.toBytes(outTuple); + ClientProtos.SerializedResultSet.Builder serializedResBuilder = ClientProtos.SerializedResultSet.newBuilder(); + serializedResBuilder.addSerializedTuples(ByteString.copyFrom(serializedBytes)); + serializedResBuilder.setSchema(schema.getProto()); + serializedResBuilder.setBytesNum(serializedBytes.length); + + responseBuilder.setResultSet(serializedResBuilder); + responseBuilder.setMaxRowNum(1); + responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto()); + responseBuilder.setResultCode(ClientProtos.ResultCode.OK); + } + } finally { + // stop script executor + stopScriptExecutors(evalContext); + } + } + + @VisibleForTesting + public static void startScriptExecutors(QueryContext queryContext, EvalContext evalContext, Target[] targets) + throws IOException { for (int i = 0; i < targets.length; i++) { EvalNode eval = targets[i].getEvalTree(); - eval.bind(null); - outTuple.put(i, eval.eval(null)); + if (eval instanceof GeneralFunctionEval) { + GeneralFunctionEval functionEval = (GeneralFunctionEval) eval; + if (functionEval.getFuncDesc().getInvocation().hasPython()) { + PythonScriptExecutor scriptExecutor = new PythonScriptExecutor(functionEval.getFuncDesc()); + evalContext.addScriptExecutor(eval, scriptExecutor); + scriptExecutor.start(queryContext); + } + } } - boolean isInsert = rootNode.getChild() != null && rootNode.getChild().getType() == NodeType.INSERT; - if (isInsert) { - InsertNode insertNode = rootNode.getChild(); - insertNonFromQuery(queryContext, insertNode, responseBuilder); - } else { - Schema schema = PlannerUtil.targetToSchema(targets); - RowStoreUtil.RowStoreEncoder encoder = RowStoreUtil.createEncoder(schema); - byte[] serializedBytes = encoder.toBytes(outTuple); - ClientProtos.SerializedResultSet.Builder serializedResBuilder = ClientProtos.SerializedResultSet.newBuilder(); - serializedResBuilder.addSerializedTuples(ByteString.copyFrom(serializedBytes)); - serializedResBuilder.setSchema(schema.getProto()); - serializedResBuilder.setBytesNum(serializedBytes.length); - - responseBuilder.setResultSet(serializedResBuilder); - responseBuilder.setMaxRowNum(1); - responseBuilder.setQueryId(QueryIdFactory.NULL_QUERY_ID.getProto()); - responseBuilder.setResultCode(ClientProtos.ResultCode.OK); + } + + @VisibleForTesting + public static void stopScriptExecutors(EvalContext evalContext) throws IOException { + for (ScriptExecutor executor : evalContext.getAllScriptExecutors()) { + executor.shutdown(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index c273fe9cbf..dd188078f9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -21,7 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Maps; - +import io.netty.handler.codec.http.QueryStringDecoder; import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,26 +30,27 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.tajo.TaskAttemptId; import org.apache.tajo.TajoProtos; import org.apache.tajo.TajoProtos.TaskAttemptState; +import org.apache.tajo.TaskAttemptId; import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.TableDesc; import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.statistics.TableStats; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.function.python.PythonScriptExecutor; -import org.apache.tajo.plan.serder.LogicalNodeDeserializer; -import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.engine.planner.physical.PhysicalExec; import org.apache.tajo.engine.query.QueryContext; import org.apache.tajo.engine.query.TaskRequest; import org.apache.tajo.ipc.QueryMasterProtocol; import org.apache.tajo.ipc.TajoWorkerProtocol.*; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; +import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.PythonScriptExecutor; +import org.apache.tajo.plan.function.python.ScriptExecutor; import org.apache.tajo.plan.logical.*; +import org.apache.tajo.plan.serder.LogicalNodeDeserializer; +import org.apache.tajo.plan.util.PlannerUtil; import org.apache.tajo.pullserver.TajoPullServerService; import org.apache.tajo.pullserver.retriever.FileChunk; import org.apache.tajo.rpc.NettyClientBase; @@ -57,8 +58,6 @@ import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.NetUtils; - -import io.netty.handler.codec.http.QueryStringDecoder; import org.apache.tajo.util.QueryContextUtil; import java.io.File; @@ -140,7 +139,7 @@ public Task(String taskRunnerId, public void initPlan() throws IOException { QueryContextUtil.updatePythonScriptPath(systemConf, queryContext); - plan = LogicalNodeDeserializer.deserialize(queryContext, request.getPlan()); + plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); if (scanNode != null) { for (LogicalNode node : scanNode) { @@ -178,7 +177,7 @@ public void initPlan() throws IOException { LOG.info("=================================="); LOG.info("* Stage " + request.getId() + " is initialized"); LOG.info("* InterQuery: " + interQuery - + (interQuery ? ", Use " + this.shuffleType + " shuffle":"") + + + (interQuery ? ", Use " + this.shuffleType + " shuffle" : "") + ", Fragments (num: " + request.getFragments().size() + ")" + ", Fetches (total:" + request.getFetches().size() + ") :"); @@ -195,8 +194,27 @@ public void initPlan() throws IOException { LOG.info("=================================="); } + private void startScriptExecutors() throws IOException { + int cnt = 0; + for (ScriptExecutor executor : context.getEvalContext().getAllScriptExecutors()) { + executor.start(queryContext); + cnt++; + } + LOG.info(cnt + " script executors are started."); + } + + private void stopScriptExecutors() throws IOException { + int cnt = 0; + for (ScriptExecutor executor : context.getEvalContext().getAllScriptExecutors()) { + executor.shutdown(); + cnt++; + } + LOG.info(cnt + " script executors are shutdowned."); + } + public void init() throws IOException { initPlan(); + startScriptExecutors(); if (context.getState() == TaskAttemptState.TA_PENDING) { // initialize a task temporal dir @@ -262,11 +280,21 @@ public void fetch() { } public void kill() { + try { + stopScriptExecutors(); + } catch (IOException e) { + throw new RuntimeException(e); + } context.setState(TaskAttemptState.TA_KILLED); context.stop(); } public void abort() { + try { + stopScriptExecutors(); + } catch (IOException e) { + throw new RuntimeException(e); + } context.stop(); } @@ -415,6 +443,7 @@ public void run() throws Exception { } catch (Throwable e) { error = e ; LOG.error(e.getMessage(), e); + stopScriptExecutors(); context.stop(); } finally { if (executor != null) { @@ -473,6 +502,7 @@ public void run() throws Exception { + ", failed: " + executionBlockContext.failedTasksNum.intValue()); cleanupTask(); } finally { + stopScriptExecutors(); executionBlockContext.releaseConnection(client); } } @@ -638,6 +668,11 @@ public void run() { if (retryNum == maxRetryNum) { LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); } + try { + stopScriptExecutors(); + } catch (IOException e) { + throw new RuntimeException(e); + } context.stop(); // retry task ctx.getFetchLatch().countDown(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java index 706e9b87f9..58028ac014 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java @@ -32,6 +32,7 @@ import org.apache.tajo.engine.planner.enforce.Enforcer; import org.apache.tajo.engine.planner.global.DataChannel; import org.apache.tajo.engine.query.QueryContext; +import org.apache.tajo.plan.expr.EvalContext; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.storage.HashShuffleAppenderManager; import org.apache.tajo.storage.fragment.FileFragment; @@ -83,6 +84,8 @@ public class TaskAttemptContext { private Map partitionOutputVolume; private HashShuffleAppenderManager hashShuffleAppenderManager; + private EvalContext evalContext = new EvalContext(); + public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext executionBlockContext, final TaskAttemptId queryId, final FragmentProto[] fragments, @@ -403,4 +406,8 @@ public TaskAttemptId getQueryId() { public HashShuffleAppenderManager getHashShuffleAppenderManager() { return hashShuffleAppenderManager; } + + public EvalContext getEvalContext() { + return evalContext; + } } \ No newline at end of file diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 80f32b16fc..1fea8a9eda 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -37,7 +37,9 @@ import org.apache.tajo.engine.json.CoreGsonHelper; import org.apache.tajo.engine.parser.SQLAnalyzer; import org.apache.tajo.function.FunctionSignature; +import org.apache.tajo.master.exec.QueryExecutor; import org.apache.tajo.plan.*; +import org.apache.tajo.plan.expr.EvalContext; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.plan.serder.EvalNodeDeserializer; import org.apache.tajo.plan.serder.EvalNodeSerializer; @@ -131,8 +133,8 @@ public TajoConf getConf() { * @return * @throws PlanningException */ - private static Target[] getRawTargets(QueryContext context, String query, boolean condition) throws PlanningException, - InvalidStatementException { + private static Target[] getRawTargets(QueryContext context, EvalContext evalContext, String query, boolean condition) + throws PlanningException, InvalidStatementException { List parsedResults = SimpleParser.parseScript(query); if (parsedResults.size() > 1) { @@ -147,7 +149,7 @@ private static Target[] getRawTargets(QueryContext context, String query, boolea } assertFalse(state.getErrorMessages().get(0), true); } - LogicalPlan plan = planner.createPlan(context, expr, true); + LogicalPlan plan = planner.createPlan(context, evalContext, expr, true); optimizer.optimize(context, plan); annotatedPlanVerifier.verify(context, state, plan); @@ -271,15 +273,17 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S Target [] targets; TajoClassLoader classLoader = new TajoClassLoader(); + EvalContext evalContext = new EvalContext(); try { - targets = getRawTargets(queryContext, query, condition); + targets = getRawTargets(queryContext, evalContext, query, condition); EvalCodeGenerator codegen = null; if (queryContext.getBool(SessionVars.CODEGEN)) { codegen = new EvalCodeGenerator(classLoader); } + QueryExecutor.startScriptExecutors(queryContext, evalContext, targets); Tuple outTuple = new VTuple(targets.length); for (int i = 0; i < targets.length; i++) { EvalNode eval = targets[i].getEvalTree(); @@ -287,7 +291,7 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S if (queryContext.getBool(SessionVars.CODEGEN)) { eval = codegen.compile(inputSchema, eval); } - eval.bind(inputSchema); + eval.bind(evalContext, inputSchema); outTuple.put(i, eval.eval(vtuple)); } @@ -324,11 +328,12 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S if (schema != null) { cat.dropTable(qualifiedTableName); } + QueryExecutor.stopScriptExecutors(evalContext); } } public static void assertEvalTreeProtoSerDer(OverridableConf context, EvalNode evalNode) { PlanProto.EvalNodeTree converted = EvalNodeSerializer.serialize(evalNode); - assertEquals(evalNode, EvalNodeDeserializer.deserialize(context, converted)); + assertEquals(evalNode, EvalNodeDeserializer.deserialize(context, null, converted)); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java index a2d05989b5..ea42783ffb 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTree.java @@ -48,7 +48,7 @@ public void testTupleEval() throws CloneNotSupportedException { schema1.addColumn("table1.score", INT4); BinaryEval expr = new BinaryEval(EvalType.PLUS, e1, e2); - expr.bind(schema1); + expr.bind(null, schema1); assertCloneEqual(expr); VTuple tuple = new VTuple(2); @@ -161,19 +161,19 @@ public void testAndTest() { MockFalseExpr falseExpr = new MockFalseExpr(); BinaryEval andExpr = new BinaryEval(EvalType.AND, trueExpr, trueExpr); - andExpr.bind(null); + andExpr.bind(null, null); assertTrue(andExpr.eval(null).asBool()); andExpr = new BinaryEval(EvalType.AND, falseExpr, trueExpr); - andExpr.bind(null); + andExpr.bind(null, null); assertFalse(andExpr.eval(null).asBool()); andExpr = new BinaryEval(EvalType.AND, trueExpr, falseExpr); - andExpr.bind(null); + andExpr.bind(null, null); assertFalse(andExpr.eval(null).asBool()); andExpr = new BinaryEval(EvalType.AND, falseExpr, falseExpr); - andExpr.bind(null); + andExpr.bind(null, null); assertFalse(andExpr.eval(null).asBool()); } @@ -183,19 +183,19 @@ public void testOrTest() { MockFalseExpr falseExpr = new MockFalseExpr(); BinaryEval orExpr = new BinaryEval(EvalType.OR, trueExpr, trueExpr); - orExpr.bind(null); + orExpr.bind(null, null); assertTrue(orExpr.eval(null).asBool()); orExpr = new BinaryEval(EvalType.OR, falseExpr, trueExpr); - orExpr.bind(null); + orExpr.bind(null, null); assertTrue(orExpr.eval(null).asBool()); orExpr = new BinaryEval(EvalType.OR, trueExpr, falseExpr); - orExpr.bind(null); + orExpr.bind(null, null); assertTrue(orExpr.eval(null).asBool()); orExpr = new BinaryEval(EvalType.OR, falseExpr, falseExpr); - orExpr.bind(null); + orExpr.bind(null, null); assertFalse(orExpr.eval(null).asBool()); } @@ -209,41 +209,41 @@ public final void testCompOperator() { e1 = new ConstEval(DatumFactory.createInt4(9)); e2 = new ConstEval(DatumFactory.createInt4(34)); expr = new BinaryEval(EvalType.LTH, e1, e2); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.LEQ, e1, e2); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.LTH, e2, e1); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.LEQ, e2, e1); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GTH, e2, e1); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GEQ, e2, e1); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GTH, e1, e2); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GEQ, e1, e2); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); BinaryEval plus = new BinaryEval(EvalType.PLUS, e1, e2); expr = new BinaryEval(EvalType.LTH, e1, plus); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.LEQ, e1, plus); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.LTH, plus, e1); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.LEQ, plus, e1); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GTH, plus, e1); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GEQ, plus, e1); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GTH, e1, plus); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); expr = new BinaryEval(EvalType.GEQ, e1, plus); - assertFalse(expr.bind(null).eval(null).asBool()); + assertFalse(expr.bind(null, null).eval(null).asBool()); } @Test @@ -256,28 +256,28 @@ public final void testArithmaticsOperator() e1 = new ConstEval(DatumFactory.createInt4(9)); e2 = new ConstEval(DatumFactory.createInt4(34)); BinaryEval expr = new BinaryEval(EvalType.PLUS, e1, e2); - assertEquals(expr.bind(null).eval(null).asInt4(), 43); + assertEquals(expr.bind(null, null).eval(null).asInt4(), 43); assertCloneEqual(expr); // MINUS e1 = new ConstEval(DatumFactory.createInt4(5)); e2 = new ConstEval(DatumFactory.createInt4(2)); expr = new BinaryEval(EvalType.MINUS, e1, e2); - assertEquals(expr.bind(null).eval(null).asInt4(), 3); + assertEquals(expr.bind(null, null).eval(null).asInt4(), 3); assertCloneEqual(expr); // MULTIPLY e1 = new ConstEval(DatumFactory.createInt4(5)); e2 = new ConstEval(DatumFactory.createInt4(2)); expr = new BinaryEval(EvalType.MULTIPLY, e1, e2); - assertEquals(expr.bind(null).eval(null).asInt4(), 10); + assertEquals(expr.bind(null, null).eval(null).asInt4(), 10); assertCloneEqual(expr); // DIVIDE e1 = new ConstEval(DatumFactory.createInt4(10)); e2 = new ConstEval(DatumFactory.createInt4(5)); expr = new BinaryEval(EvalType.DIVIDE, e1, e2); - assertEquals(expr.bind(null).eval(null).asInt4(), 2); + assertEquals(expr.bind(null, null).eval(null).asInt4(), 2); assertCloneEqual(expr); } @@ -293,7 +293,7 @@ public final void testGetReturnType() { assertEquals(CatalogUtil.newSimpleDataType(INT4), expr.getValueType()); expr = new BinaryEval(EvalType.LTH, e1, e2); - assertTrue(expr.bind(null).eval(null).asBool()); + assertTrue(expr.bind(null, null).eval(null).asBool()); assertEquals(CatalogUtil.newSimpleDataType(BOOLEAN), expr.getValueType()); e1 = new ConstEval(DatumFactory.createFloat8(9.3)); @@ -384,7 +384,7 @@ public final void testBindCheck() { binEval.eval(null); fail("EvalNode is not binded"); } catch (IllegalStateException e) { - assertTrue(binEval.bind(null).eval(null).asBool()); + assertTrue(binEval.bind(null, null).eval(null).asBool()); } CaseWhenEval caseWhenEval = new CaseWhenEval(); @@ -393,7 +393,7 @@ public final void testBindCheck() { caseWhenEval.eval(null); fail("EvalNode is not binded"); } catch (IllegalStateException e) { - assertEquals(caseWhenEval.bind(null).eval(null).asInt4(), 1); + assertEquals(caseWhenEval.bind(null, null).eval(null).asInt4(), 1); } Schema schema = new Schema(new Column[]{new Column("test", TajoDataTypes.Type.INT4)}); @@ -404,7 +404,7 @@ public final void testBindCheck() { regexEval.eval(null); fail("EvalNode is not binded"); } catch (IllegalStateException e) { - assertEquals(regexEval.bind(schema).eval(tuple).asBool(), true); + assertEquals(regexEval.bind(null, schema).eval(tuple).asBool(), true); } RowConstantEval rowConstantEval = new RowConstantEval(new Datum[]{}); @@ -412,7 +412,7 @@ public final void testBindCheck() { rowConstantEval.eval(null); fail("EvalNode is not binded"); } catch (IllegalStateException e) { - assertEquals(rowConstantEval.bind(null).eval(null).isNull(), true); + assertEquals(rowConstantEval.bind(null, null).eval(null).isNull(), true); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java index e23a34bdcd..6cad17b88a 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java @@ -136,11 +136,11 @@ public static void tearDown() throws Exception { util.shutdownCatalogCluster(); } - public static Target [] getRawTargets(String query) { + public static Target [] getRawTargets(EvalContext evalContext, String query) { Expr expr = analyzer.parse(query); LogicalPlan plan = null; try { - plan = planner.createPlan(defaultContext, expr); + plan = planner.createPlan(defaultContext, evalContext, expr); } catch (PlanningException e) { e.printStackTrace(); } @@ -159,7 +159,7 @@ public static EvalNode getRootSelection(String query) throws PlanningException { } LogicalPlanner.PlanContext context = new LogicalPlanner.PlanContext(defaultContext, plan, plan.getRootBlock(), - new EvalTreeOptimizer(), true); + new EvalTreeOptimizer(null), true); Selection selection = plan.getRootBlock().getSingletonExpr(OpType.Filter); return planner.getExprAnnotator().createEvalNode(context, selection.getQual(), @@ -265,12 +265,12 @@ public final void testGetCNF() throws PlanningException { FieldEval field = first.getLeftExpr(); assertEquals(col1, field.getColumnRef()); assertEquals(EvalType.LTH, first.getType()); - assertEquals(10, first.getRightExpr().bind(null).eval(null).asInt4()); + assertEquals(10, first.getRightExpr().bind(null, null).eval(null).asInt4()); field = second.getRightExpr(); assertEquals(col1, field.getColumnRef()); assertEquals(EvalType.LTH, second.getType()); - assertEquals(4, second.getLeftExpr().bind(null).eval(null).asInt4()); + assertEquals(4, second.getLeftExpr().bind(null, null).eval(null).asInt4()); } @Test @@ -304,13 +304,13 @@ public final void testSimplify() throws PlanningException { Target [] targets = getRawTargets(QUERIES[0]); EvalNode node = AlgebraicUtil.eliminateConstantExprs(targets[0].getEvalTree()); assertEquals(EvalType.CONST, node.getType()); - assertEquals(7, node.bind(null).eval(null).asInt4()); + assertEquals(7, node.bind(null, null).eval(null).asInt4()); node = AlgebraicUtil.eliminateConstantExprs(targets[1].getEvalTree()); assertEquals(EvalType.CONST, node.getType()); - assertTrue(7.0d == node.bind(null).eval(null).asFloat8()); + assertTrue(7.0d == node.bind(null, null).eval(null).asFloat8()); Expr expr = analyzer.parse(QUERIES[1]); - LogicalPlan plan = planner.createPlan(defaultContext, expr, true); + LogicalPlan plan = planner.createPlan(defaultContext, new EvalContext(), expr, true); targets = plan.getRootBlock().getRawTargets(); Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4); Collection exprs = @@ -335,7 +335,7 @@ public final void testTranspose() throws PlanningException { assertEquals(EvalType.GTH, transposed.getType()); FieldEval field = transposed.getLeftExpr(); assertEquals(col1, field.getColumnRef()); - assertEquals(1, transposed.getRightExpr().bind(null).eval(null).asInt4()); + assertEquals(1, transposed.getRightExpr().bind(null, null).eval(null).asInt4()); node = getRootSelection(QUERIES[4]); // we expect that score < 3 @@ -343,7 +343,7 @@ public final void testTranspose() throws PlanningException { assertEquals(EvalType.LTH, transposed.getType()); field = transposed.getLeftExpr(); assertEquals(col1, field.getColumnRef()); - assertEquals(2, transposed.getRightExpr().bind(null).eval(null).asInt4()); + assertEquals(2, transposed.getRightExpr().bind(null, null).eval(null).asInt4()); } @Test diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index e9638d2002..3db7038e09 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -72,6 +72,7 @@ static class Context { LogicalPlan plan; LogicalPlan.QueryBlock currentBlock; NameResolvingMode columnRsvLevel; + EvalContext evalContext; public Context(LogicalPlanner.PlanContext planContext, NameResolvingMode colRsvLevel) { this.queryContext = planContext.queryContext; @@ -384,7 +385,7 @@ public EvalNode visitValueListExpr(Context ctx, Stack stack, ValueListExpr if (!EvalTreeUtil.checkIfCanBeConstant(evalNodes[i])) { throw new PlanningException("Non constant values cannot be included in IN PREDICATE."); } - values[i] = EvalTreeUtil.evaluateImmediately(evalNodes[i]); + values[i] = EvalTreeUtil.evaluateImmediately(ctx.evalContext, evalNodes[i]); } return new RowConstantEval(values); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java index d7c631b4c1..3bf5db75e0 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java @@ -86,9 +86,10 @@ public static class PlanContext { EvalTreeOptimizer evalOptimizer; TimeZone timeZone; boolean debugOrUnitTests; + EvalContext evalContext; public PlanContext(OverridableConf context, LogicalPlan plan, QueryBlock block, EvalTreeOptimizer evalOptimizer, - boolean debugOrUnitTests) { + EvalContext evalContext, boolean debugOrUnitTests) { this.queryContext = context; this.plan = plan; this.queryBlock = block; @@ -100,6 +101,7 @@ public PlanContext(OverridableConf context, LogicalPlan plan, QueryBlock block, timeZone = TimeZone.getTimeZone(timezoneId); } + this.evalContext = evalContext; this.debugOrUnitTests = debugOrUnitTests; } @@ -109,12 +111,17 @@ public PlanContext(PlanContext context, QueryBlock block) { this.queryBlock = block; this.evalOptimizer = context.evalOptimizer; this.debugOrUnitTests = context.debugOrUnitTests; + this.evalContext = context.evalContext; } public QueryBlock getQueryBlock() { return queryBlock; } + public EvalContext getEvalContext() { + return evalContext; + } + public String toString() { return "block=" + queryBlock.getName() + ", relNum=" + queryBlock.getRelations().size() + ", "+ queryBlock.namedExprsMgr.toString(); @@ -127,17 +134,18 @@ public String toString() { * @param expr A relational algebraic expression for a query. * @return A logical plan */ - public LogicalPlan createPlan(OverridableConf context, Expr expr) throws PlanningException { - return createPlan(context, expr, false); + public LogicalPlan createPlan(OverridableConf context, EvalContext evalContext, Expr expr) throws PlanningException { + return createPlan(context, evalContext, expr, false); } @VisibleForTesting - public LogicalPlan createPlan(OverridableConf queryContext, Expr expr, boolean debug) throws PlanningException { + public LogicalPlan createPlan(OverridableConf queryContext, EvalContext evalContext, Expr expr, boolean debug) + throws PlanningException { LogicalPlan plan = new LogicalPlan(this); QueryBlock rootBlock = plan.newAndGetBlock(LogicalPlan.ROOT_BLOCK); - PlanContext context = new PlanContext(queryContext, plan, rootBlock, evalOptimizer, debug); + PlanContext context = new PlanContext(queryContext, plan, rootBlock, evalOptimizer, evalContext, debug); preprocessor.visit(context, new Stack(), expr); plan.resetGeneratedId(); LogicalNode topMostNode = this.visit(context, new Stack(), expr); @@ -810,7 +818,7 @@ public LimitNode visitLimit(PlanContext context, Stack stack, Limit limit) limitNode.setInSchema(child.getOutSchema()); limitNode.setOutSchema(child.getOutSchema()); - firstFetNum.bind(null); + firstFetNum.bind(null, null); limitNode.setFetchFirst(firstFetNum.eval(null).asInt8()); return limitNode; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java index fb05f33b72..6cf72723e9 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/AlgebraicUtil.java @@ -149,7 +149,7 @@ public EvalNode visitBinaryEval(Object context, Stack stack, BinaryEva } if (lhs.getType() == EvalType.CONST && rhs.getType() == EvalType.CONST) { - return new ConstEval(binaryEval.bind(null).eval(null)); + return new ConstEval(binaryEval.bind(null, null).eval(null)); } return binaryEval; @@ -162,7 +162,7 @@ public EvalNode visitUnaryEval(Object context, Stack stack, UnaryEval stack.pop(); if (child.getType() == EvalType.CONST) { - return new ConstEval(unaryEval.bind(null).eval(null)); + return new ConstEval(unaryEval.bind(null, null).eval(null)); } return unaryEval; @@ -184,7 +184,7 @@ public EvalNode visitFuncCall(Object context, FunctionEval evalNode, Stack scriptExecutorMap = TUtil.newHashMap(); + + public void addScriptExecutor(EvalNode evalNode, ScriptExecutor scriptExecutor) { + this.scriptExecutorMap.put(evalNode, scriptExecutor); + } + + public boolean hasScriptExecutor(EvalNode evalNode) { + return this.scriptExecutorMap.containsKey(evalNode); + } + + public ScriptExecutor getScriptExecutor(EvalNode evalNode) { + return this.scriptExecutorMap.get(evalNode); + } + + public Collection getAllScriptExecutors() { + return this.scriptExecutorMap.values(); + } +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java index 154b0fd8a7..9abb0bc72d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalNode.java @@ -19,6 +19,7 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.ProtoObject; import org.apache.tajo.common.TajoDataTypes.DataType; @@ -61,9 +62,9 @@ public String toJson() { return PlanGsonHelper.toJson(this, EvalNode.class); } - public EvalNode bind(Schema schema) { + public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { for (int i = 0; i < childNum(); i++) { - getChild(i).bind(schema); + getChild(i).bind(evalContext, schema); } isBinded = true; return this; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java index 5e3843c797..1fa2fe090d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalTreeUtil.java @@ -570,8 +570,8 @@ public static boolean checkIfCanBeConstant(EvalNode evalNode) { return findUniqueColumns(evalNode).size() == 0 && findDistinctAggFunction(evalNode).size() == 0; } - public static Datum evaluateImmediately(EvalNode evalNode) { - evalNode.bind(null); + public static Datum evaluateImmediately(EvalContext evalContext, EvalNode evalNode) { + evalNode.bind(evalContext, null); return evalNode.eval(null); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java index 83d00b94fa..870970b538 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java @@ -40,8 +40,8 @@ public FieldEval(Column column) { } @Override - public EvalNode bind(Schema schema) { - super.bind(schema); + public EvalNode bind(EvalContext evalContext, Schema schema) { + super.bind(evalContext, schema); // TODO - column namespace should be improved to simplify name handling and resolving. if (column.hasQualifier()) { fieldId = schema.getColumnId(column.getQualifiedName()); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java index 4ff754845b..dd9121bd5a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java @@ -49,8 +49,8 @@ public FunctionEval(EvalType type, FunctionDesc funcDesc, EvalNode[] argEvals) { } @Override - public EvalNode bind(Schema schema) { - super.bind(schema); + public EvalNode bind(EvalContext evalContext, Schema schema) { + super.bind(evalContext, schema); this.params = new VTuple(argEvals.length); return this; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index fa9aca3537..2e15534413 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -40,10 +40,13 @@ public GeneralFunctionEval(OverridableConf queryContext, FunctionDesc desc, Eval } @Override - public EvalNode bind(Schema schema) { - super.bind(schema); + public EvalNode bind(EvalContext evalContext, Schema schema) { + super.bind(evalContext, schema); try { this.funcInvoke = FunctionInvoke.newInstance(funcDesc); + if (evalContext != null && evalContext.hasScriptExecutor(this)) { + this.invokeContext.setScriptExecutor(evalContext.getScriptExecutor(this)); + } this.funcInvoke.init(invokeContext); } catch (IOException e) { throw new RuntimeException(e); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java index 6faa6679f0..cdd8dfb575 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java @@ -74,8 +74,8 @@ public String getName() { } @Override - public EvalNode bind(Schema schema) { - super.bind(schema); + public EvalNode bind(EvalContext evalContext, Schema schema) { + super.bind(evalContext, schema); compile(pattern); return this; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizationRule.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizationRule.java index 7e0322475a..2440c52b82 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizationRule.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizationRule.java @@ -20,9 +20,10 @@ import org.apache.tajo.plan.LogicalPlanner; import org.apache.tajo.plan.annotator.Prioritized; +import org.apache.tajo.plan.expr.EvalContext; import org.apache.tajo.plan.expr.EvalNode; @Prioritized public interface EvalTreeOptimizationRule { - public EvalNode optimize(LogicalPlanner.PlanContext context, EvalNode tree); + EvalNode optimize(LogicalPlanner.PlanContext context, EvalNode tree); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizer.java index a8a3ff3c51..680600a743 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/EvalTreeOptimizer.java @@ -24,6 +24,7 @@ import org.apache.commons.logging.LogFactory; import org.apache.tajo.plan.LogicalPlanner; import org.apache.tajo.plan.annotator.Prioritized; +import org.apache.tajo.plan.expr.EvalContext; import org.apache.tajo.plan.expr.EvalNode; import org.apache.tajo.util.ClassUtil; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index 0356ad50df..df0b92d364 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -51,7 +51,7 @@ public EvalNode visitBinaryEval(LogicalPlanner.PlanContext context, Stack(1); this.outputQueue = new ArrayBlockingQueue(2); startUdfController(); createInputHandlers(); setStreams(); startThreads(); + LOG.info("started"); } - public void shutdown() throws IOException, InterruptedException { + public void shutdown() throws IOException { isStopped = true; process.destroy(); if (stdin != null) { @@ -156,7 +160,7 @@ public void shutdown() throws IOException, InterruptedException { private StreamingCommand startUdfController() throws IOException { StreamingCommand sc = new StreamingCommand(buildCommand()); - ProcessBuilder processBuilder = StreamingUtil.createProcess(invokeContext.getQueryContext(), sc); + ProcessBuilder processBuilder = StreamingUtil.createProcess(queryContext, sc); process = processBuilder.start(); Runtime.getRuntime().addShutdownHook(new ProcessKiller()); @@ -170,14 +174,13 @@ private StreamingCommand startUdfController() throws IOException { * @throws IOException */ private String[] buildCommand() throws IOException { - OverridableConf queryContext = invokeContext.getQueryContext(); String[] command = new String[10]; // TODO: support controller logging String standardOutputRootWriteLocation = ""; if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { LOG.warn("Currently, logging is not supported for the python controller."); - standardOutputRootWriteLocation = invokeContext.getQueryContext().get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); + standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); } // standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; @@ -198,10 +201,10 @@ private String[] buildCommand() throws IOException { command[UDF_FILE_NAME] = fileName; command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; - if (!invokeContext.getQueryContext().containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { + if (!queryContext.containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); } - String fileCachePath = invokeContext.getQueryContext().get(QueryVars.PYTHON_SCRIPT_CODE_DIR); + String fileCachePath = queryContext.get(QueryVars.PYTHON_SCRIPT_CODE_DIR); command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; command[STD_OUT_OUTPUT_PATH] = outFileName; command[STD_ERR_OUTPUT_PATH] = errOutFileName; @@ -455,8 +458,6 @@ public void run() { shutdown(); } catch (IOException e) { throw new RuntimeException(e); - } catch (InterruptedException e) { - throw new RuntimeException(e); } } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java new file mode 100644 index 0000000000..3a8fe2ab0b --- /dev/null +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java @@ -0,0 +1,14 @@ +package org.apache.tajo.plan.function.python; + +import org.apache.tajo.OverridableConf; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.storage.Tuple; + +import java.io.IOException; + +public interface ScriptExecutor { +// void start(FunctionInvokeContext context) throws IOException; + void start(OverridableConf queryContext) throws IOException; + void shutdown() throws IOException; + Datum eval(Tuple input); +} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java index 8a24add6c1..43a86184b3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java @@ -48,7 +48,7 @@ public boolean isEligible(OverridableConf queryContext, LogicalPlan plan) { public LogicalPlan rewrite(OverridableConf queryContext, LogicalPlan plan) throws PlanningException { LogicalNode root = plan.getRootBlock().getRoot(); PlanProto.LogicalNodeTree serialized = LogicalNodeSerializer.serialize(plan.getRootBlock().getRoot()); - LogicalNode deserialized = LogicalNodeDeserializer.deserialize(queryContext, serialized); + LogicalNode deserialized = LogicalNodeDeserializer.deserialize(queryContext, null, serialized); assert root.deepEquals(deserialized); return plan; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java index c49d51b134..d8b638042a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/PartitionedTableRewriter.java @@ -86,7 +86,7 @@ private static class PartitionPathFilter implements PathFilter { public PartitionPathFilter(Schema schema, EvalNode partitionFilter) { this.schema = schema; this.partitionFilter = partitionFilter; - partitionFilter.bind(schema); + partitionFilter.bind(null, schema); } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 0a60f80ea6..0cf476def0 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -35,6 +35,7 @@ import org.apache.tajo.exception.InternalException; import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.function.AggFunction; +import org.apache.tajo.plan.function.python.PythonScriptExecutor; import org.apache.tajo.plan.logical.WindowSpec; import org.apache.tajo.plan.serder.PlanProto.WinFunctionEvalSpec; @@ -52,7 +53,7 @@ */ public class EvalNodeDeserializer { - public static EvalNode deserialize(OverridableConf context, PlanProto.EvalNodeTree tree) { + public static EvalNode deserialize(OverridableConf context, EvalContext evalContext, PlanProto.EvalNodeTree tree) { Map evalNodeMap = Maps.newHashMap(); // sort serialized eval nodes in an ascending order of their IDs. @@ -180,7 +181,10 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { try { funcDesc = new FunctionDesc(funcProto.getFuncion()); if (type == EvalType.FUNCTION) { - current = new GeneralFunctionEval(context, new FunctionDesc(funcProto.getFuncion()), params); + current = new GeneralFunctionEval(context, funcDesc, params); + if (funcDesc.getInvocation().hasPython()) { + evalContext.addScriptExecutor(current, new PythonScriptExecutor(funcDesc)); + } } else if (type == EvalType.AGG_FUNCTION || type == EvalType.WINDOW_FUNCTION) { AggFunction instance = (AggFunction) funcDesc.newInstance(); if (type == EvalType.AGG_FUNCTION) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java index f96626baa4..84991bbf75 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path; import org.apache.tajo.OverridableConf; import org.apache.tajo.algebra.JoinType; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.Schema; import org.apache.tajo.catalog.SortSpec; @@ -31,10 +32,7 @@ import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.exception.UnimplementedException; import org.apache.tajo.plan.Target; -import org.apache.tajo.plan.expr.AggregationFunctionCallEval; -import org.apache.tajo.plan.expr.EvalNode; -import org.apache.tajo.plan.expr.FieldEval; -import org.apache.tajo.plan.expr.WindowFunctionEval; +import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.logical.*; import org.apache.tajo.util.KeyValueSet; import org.apache.tajo.util.TUtil; @@ -58,7 +56,8 @@ public class LogicalNodeDeserializer { * @param tree LogicalNodeTree which contains a list of serialized logical nodes. * @return A logical node tree */ - public static LogicalNode deserialize(OverridableConf context, PlanProto.LogicalNodeTree tree) { + public static LogicalNode deserialize(OverridableConf context, @Nullable EvalContext evalContext, + PlanProto.LogicalNodeTree tree) { Map nodeMap = Maps.newHashMap(); // sort serialized logical nodes in an ascending order of their sids @@ -88,10 +87,10 @@ public int compare(PlanProto.LogicalNode o1, PlanProto.LogicalNode o2) { current = convertSetSession(protoNode); break; case EXPRS: - current = convertEvalExpr(context, protoNode); + current = convertEvalExpr(context, evalContext, protoNode); break; case PROJECTION: - current = convertProjection(context, nodeMap, protoNode); + current = convertProjection(context, evalContext, nodeMap, protoNode); break; case LIMIT: current = convertLimit(nodeMap, protoNode); @@ -100,34 +99,34 @@ public int compare(PlanProto.LogicalNode o1, PlanProto.LogicalNode o2) { current = convertSort(nodeMap, protoNode); break; case WINDOW_AGG: - current = convertWindowAgg(context, nodeMap, protoNode); + current = convertWindowAgg(context, evalContext, nodeMap, protoNode); break; case HAVING: - current = convertHaving(context, nodeMap, protoNode); + current = convertHaving(context, evalContext, nodeMap, protoNode); break; case GROUP_BY: - current = convertGroupby(context, nodeMap, protoNode); + current = convertGroupby(context, evalContext, nodeMap, protoNode); break; case DISTINCT_GROUP_BY: - current = convertDistinctGroupby(context, nodeMap, protoNode); + current = convertDistinctGroupby(context, evalContext, nodeMap, protoNode); break; case SELECTION: - current = convertFilter(context, nodeMap, protoNode); + current = convertFilter(context, evalContext, nodeMap, protoNode); break; case JOIN: - current = convertJoin(context, nodeMap, protoNode); + current = convertJoin(context, evalContext, nodeMap, protoNode); break; case TABLE_SUBQUERY: - current = convertTableSubQuery(context, nodeMap, protoNode); + current = convertTableSubQuery(context, evalContext, nodeMap, protoNode); break; case UNION: current = convertUnion(nodeMap, protoNode); break; case PARTITIONS_SCAN: - current = convertPartitionScan(context, protoNode); + current = convertPartitionScan(context, evalContext, protoNode); break; case SCAN: - current = convertScan(context, protoNode); + current = convertScan(context, evalContext, protoNode); break; case CREATE_TABLE: @@ -192,22 +191,25 @@ private static SetSessionNode convertSetSession(PlanProto.LogicalNode protoNode) return setSession; } - private static EvalExprNode convertEvalExpr(OverridableConf context, PlanProto.LogicalNode protoNode) { + private static EvalExprNode convertEvalExpr(OverridableConf context, EvalContext evalContext, + PlanProto.LogicalNode protoNode) { PlanProto.EvalExprNode evalExprProto = protoNode.getExprEval(); EvalExprNode evalExpr = new EvalExprNode(protoNode.getNodeId()); evalExpr.setInSchema(convertSchema(protoNode.getInSchema())); - evalExpr.setTargets(convertTargets(context, evalExprProto.getTargetsList())); + evalExpr.setTargets(convertTargets(context, evalContext, evalExprProto.getTargetsList())); return evalExpr; } - private static ProjectionNode convertProjection(OverridableConf context, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static ProjectionNode convertProjection(OverridableConf context, EvalContext evalContext, + Map nodeMap, + PlanProto.LogicalNode protoNode) { PlanProto.ProjectionNode projectionProto = protoNode.getProjection(); ProjectionNode projectionNode = new ProjectionNode(protoNode.getNodeId()); - projectionNode.init(projectionProto.getDistinct(), convertTargets(context, projectionProto.getTargetsList())); + projectionNode.init(projectionProto.getDistinct(), convertTargets(context, evalContext, + projectionProto.getTargetsList())); projectionNode.setChild(nodeMap.get(projectionProto.getChildSeq())); projectionNode.setInSchema(convertSchema(protoNode.getInSchema())); projectionNode.setOutSchema(convertSchema(protoNode.getOutSchema())); @@ -239,21 +241,22 @@ private static SortNode convertSort(Map nodeMap, PlanProto return sortNode; } - private static HavingNode convertHaving(OverridableConf context, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static HavingNode convertHaving(OverridableConf context, EvalContext evalContext, + Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.FilterNode havingProto = protoNode.getFilter(); HavingNode having = new HavingNode(protoNode.getNodeId()); having.setChild(nodeMap.get(havingProto.getChildSeq())); - having.setQual(EvalNodeDeserializer.deserialize(context, havingProto.getQual())); + having.setQual(EvalNodeDeserializer.deserialize(context, evalContext, havingProto.getQual())); having.setInSchema(convertSchema(protoNode.getInSchema())); having.setOutSchema(convertSchema(protoNode.getOutSchema())); return having; } - private static WindowAggNode convertWindowAgg(OverridableConf context, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static WindowAggNode convertWindowAgg(OverridableConf context, EvalContext evalContext, + Map nodeMap, + PlanProto.LogicalNode protoNode) { PlanProto.WindowAggNode windowAggProto = protoNode.getWindowAgg(); WindowAggNode windowAgg = new WindowAggNode(protoNode.getNodeId()); @@ -264,7 +267,8 @@ private static WindowAggNode convertWindowAgg(OverridableConf context, Map 0) { - windowAgg.setWindowFunctions(convertWindowFunccEvals(context, windowAggProto.getWindowFunctionsList())); + windowAgg.setWindowFunctions(convertWindowFunccEvals(context, evalContext, + windowAggProto.getWindowFunctionsList())); } windowAgg.setDistinct(windowAggProto.getDistinct()); @@ -274,7 +278,7 @@ private static WindowAggNode convertWindowAgg(OverridableConf context, Map 0) { - windowAgg.setTargets(convertTargets(context, windowAggProto.getTargetsList())); + windowAgg.setTargets(convertTargets(context, evalContext, windowAggProto.getTargetsList())); } windowAgg.setInSchema(convertSchema(protoNode.getInSchema())); @@ -283,8 +287,8 @@ private static WindowAggNode convertWindowAgg(OverridableConf context, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static GroupbyNode convertGroupby(OverridableConf context, EvalContext evalContext, + Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.GroupbyNode groupbyProto = protoNode.getGroupby(); GroupbyNode groupby = new GroupbyNode(protoNode.getNodeId()); @@ -295,10 +299,10 @@ private static GroupbyNode convertGroupby(OverridableConf context, Map 0) { - groupby.setAggFunctions(convertAggFuncCallEvals(context, groupbyProto.getAggFunctionsList())); + groupby.setAggFunctions(convertAggFuncCallEvals(context, evalContext, groupbyProto.getAggFunctionsList())); } if (groupbyProto.getTargetsCount() > 0) { - groupby.setTargets(convertTargets(context, groupbyProto.getTargetsList())); + groupby.setTargets(convertTargets(context, evalContext, groupbyProto.getTargetsList())); } groupby.setInSchema(convertSchema(protoNode.getInSchema())); @@ -307,21 +311,23 @@ private static GroupbyNode convertGroupby(OverridableConf context, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf context, EvalContext evalContext, + Map nodeMap, + PlanProto.LogicalNode protoNode) { PlanProto.DistinctGroupbyNode distinctGroupbyProto = protoNode.getDistinctGroupby(); DistinctGroupbyNode distinctGroupby = new DistinctGroupbyNode(protoNode.getNodeId()); distinctGroupby.setChild(nodeMap.get(distinctGroupbyProto.getChildSeq())); if (distinctGroupbyProto.hasGroupbyNode()) { - distinctGroupby.setGroupbyPlan(convertGroupby(context, nodeMap, distinctGroupbyProto.getGroupbyNode())); + distinctGroupby.setGroupbyPlan(convertGroupby(context, evalContext, nodeMap, + distinctGroupbyProto.getGroupbyNode())); } if (distinctGroupbyProto.getSubPlansCount() > 0) { List subPlans = TUtil.newList(); for (int i = 0; i < distinctGroupbyProto.getSubPlansCount(); i++) { - subPlans.add(convertGroupby(context, nodeMap, distinctGroupbyProto.getSubPlans(i))); + subPlans.add(convertGroupby(context, evalContext, nodeMap, distinctGroupbyProto.getSubPlans(i))); } distinctGroupby.setSubPlans(subPlans); } @@ -330,10 +336,11 @@ private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf contex distinctGroupby.setGroupingColumns(convertColumns(distinctGroupbyProto.getGroupingKeysList())); } if (distinctGroupbyProto.getAggFunctionsCount() > 0) { - distinctGroupby.setAggFunctions(convertAggFuncCallEvals(context, distinctGroupbyProto.getAggFunctionsList())); + distinctGroupby.setAggFunctions(convertAggFuncCallEvals(context, evalContext, + distinctGroupbyProto.getAggFunctionsList())); } if (distinctGroupbyProto.getTargetsCount() > 0) { - distinctGroupby.setTargets(convertTargets(context, distinctGroupbyProto.getTargetsList())); + distinctGroupby.setTargets(convertTargets(context, evalContext, distinctGroupbyProto.getTargetsList())); } int [] resultColumnIds = new int[distinctGroupbyProto.getResultIdCount()]; for (int i = 0; i < distinctGroupbyProto.getResultIdCount(); i++) { @@ -348,8 +355,8 @@ private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf contex return distinctGroupby; } - private static JoinNode convertJoin(OverridableConf context, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static JoinNode convertJoin(OverridableConf context, EvalContext evalContext, + Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.JoinNode joinProto = protoNode.getJoin(); JoinNode join = new JoinNode(protoNode.getNodeId()); @@ -359,24 +366,24 @@ private static JoinNode convertJoin(OverridableConf context, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static SelectionNode convertFilter(OverridableConf context, EvalContext evalContext, + Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.FilterNode filterProto = protoNode.getFilter(); SelectionNode selection = new SelectionNode(protoNode.getNodeId()); selection.setInSchema(convertSchema(protoNode.getInSchema())); selection.setOutSchema(convertSchema(protoNode.getOutSchema())); selection.setChild(nodeMap.get(filterProto.getChildSeq())); - selection.setQual(EvalNodeDeserializer.deserialize(context, filterProto.getQual())); + selection.setQual(EvalNodeDeserializer.deserialize(context, evalContext, filterProto.getQual())); return selection; } @@ -393,14 +400,15 @@ private static UnionNode convertUnion(Map nodeMap, PlanPro return union; } - private static ScanNode convertScan(OverridableConf context, PlanProto.LogicalNode protoNode) { + private static ScanNode convertScan(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode) { ScanNode scan = new ScanNode(protoNode.getNodeId()); - fillScanNode(context, protoNode, scan); + fillScanNode(context, evalContext, protoNode, scan); return scan; } - private static void fillScanNode(OverridableConf context, PlanProto.LogicalNode protoNode, ScanNode scan) { + private static void fillScanNode(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode, + ScanNode scan) { PlanProto.ScanNode scanProto = protoNode.getScan(); if (scanProto.hasAlias()) { scan.init(new TableDesc(scanProto.getTable()), scanProto.getAlias()); @@ -409,11 +417,11 @@ private static void fillScanNode(OverridableConf context, PlanProto.LogicalNode } if (scanProto.getExistTargets()) { - scan.setTargets(convertTargets(context, scanProto.getTargetsList())); + scan.setTargets(convertTargets(context, evalContext, scanProto.getTargetsList())); } if (scanProto.hasQual()) { - scan.setQual(EvalNodeDeserializer.deserialize(context, scanProto.getQual())); + scan.setQual(EvalNodeDeserializer.deserialize(context, evalContext, scanProto.getQual())); } if(scanProto.hasBroadcast()){ @@ -423,9 +431,10 @@ private static void fillScanNode(OverridableConf context, PlanProto.LogicalNode scan.setOutSchema(convertSchema(protoNode.getOutSchema())); } - private static PartitionedTableScanNode convertPartitionScan(OverridableConf context, PlanProto.LogicalNode protoNode) { + private static PartitionedTableScanNode convertPartitionScan(OverridableConf context, EvalContext evalContext, + PlanProto.LogicalNode protoNode) { PartitionedTableScanNode partitionedScan = new PartitionedTableScanNode(protoNode.getNodeId()); - fillScanNode(context, protoNode, partitionedScan); + fillScanNode(context, evalContext, protoNode, partitionedScan); PlanProto.PartitionScanSpec partitionScanProto = protoNode.getPartitionScan(); Path [] paths = new Path[partitionScanProto.getPathsCount()]; @@ -436,16 +445,16 @@ private static PartitionedTableScanNode convertPartitionScan(OverridableConf con return partitionedScan; } - private static TableSubQueryNode convertTableSubQuery(OverridableConf context, - Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static TableSubQueryNode convertTableSubQuery(OverridableConf context, EvalContext evalContext, + Map nodeMap, + PlanProto.LogicalNode protoNode) { PlanProto.TableSubQueryNode proto = protoNode.getTableSubQuery(); TableSubQueryNode tableSubQuery = new TableSubQueryNode(protoNode.getNodeId()); tableSubQuery.init(proto.getTableName(), nodeMap.get(proto.getChildSeq())); tableSubQuery.setInSchema(convertSchema(protoNode.getInSchema())); if (proto.getTargetsCount() > 0) { - tableSubQuery.setTargets(convertTargets(context, proto.getTargetsList())); + tableSubQuery.setTargets(convertTargets(context, evalContext, proto.getTargetsList())); } return tableSubQuery; @@ -602,20 +611,21 @@ private static TruncateTableNode convertTruncateTable(PlanProto.LogicalNode prot return truncateTable; } - private static AggregationFunctionCallEval [] convertAggFuncCallEvals(OverridableConf context, + private static AggregationFunctionCallEval [] convertAggFuncCallEvals(OverridableConf context, EvalContext evalContext, List evalTrees) { AggregationFunctionCallEval [] aggFuncs = new AggregationFunctionCallEval[evalTrees.size()]; for (int i = 0; i < aggFuncs.length; i++) { - aggFuncs[i] = (AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(context, evalTrees.get(i)); + aggFuncs[i] = (AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(context, evalContext, + evalTrees.get(i)); } return aggFuncs; } - private static WindowFunctionEval[] convertWindowFunccEvals(OverridableConf context, - List evalTrees) { + private static WindowFunctionEval[] convertWindowFunccEvals(OverridableConf context, EvalContext evalContext, + List evalTrees) { WindowFunctionEval[] winFuncEvals = new WindowFunctionEval[evalTrees.size()]; for (int i = 0; i < winFuncEvals.length; i++) { - winFuncEvals[i] = (WindowFunctionEval) EvalNodeDeserializer.deserialize(context, evalTrees.get(i)); + winFuncEvals[i] = (WindowFunctionEval) EvalNodeDeserializer.deserialize(context, evalContext, evalTrees.get(i)); } return winFuncEvals; } @@ -632,11 +642,12 @@ public static Column[] convertColumns(List columnProt return columns; } - public static Target[] convertTargets(OverridableConf context, List targetsProto) { + public static Target[] convertTargets(OverridableConf context, EvalContext evalContext, + List targetsProto) { Target [] targets = new Target[targetsProto.size()]; for (int i = 0; i < targets.length; i++) { PlanProto.Target targetProto = targetsProto.get(i); - EvalNode evalNode = EvalNodeDeserializer.deserialize(context, targetProto.getExpr()); + EvalNode evalNode = EvalNodeDeserializer.deserialize(context, evalContext, targetProto.getExpr()); if (targetProto.hasAlias()) { targets[i] = new Target(evalNode, targetProto.getAlias()); } else { From 54ed6769eb034ca48aee0b6c759def37579e7a15 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 10 Apr 2015 21:42:00 +0900 Subject: [PATCH 44/55] TAJO-1344_3 --- .../tajo/master/exec/QueryExecutor.java | 11 ++++----- .../apache/tajo/engine/eval/ExprTestBase.java | 6 ++--- .../tajo/engine/eval/TestEvalTreeUtil.java | 8 +++---- .../org/apache/tajo/plan/LogicalPlanner.java | 17 ++++++-------- .../exprrewrite/rules/ConstantFolding.java | 23 +++++++++++++++---- .../plan/serder/EvalNodeDeserializer.java | 2 +- 6 files changed, 39 insertions(+), 28 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index 8165d03e8d..247a11f06f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -89,7 +89,7 @@ public QueryExecutor(TajoMaster.MasterContext context, DDLExecutor ddlExecutor) } public SubmitQueryResponse execute(QueryContext queryContext, Session session, String sql, String jsonExpr, - LogicalPlan plan, EvalContext evalContext) throws Exception { + LogicalPlan plan) throws Exception { SubmitQueryResponse.Builder response = SubmitQueryResponse.newBuilder(); response.setIsForwarded(false); @@ -120,7 +120,7 @@ public SubmitQueryResponse execute(QueryContext queryContext, Session session, S // NonFromQuery indicates a form of 'select a, x+y;' } else if (PlannerUtil.checkIfNonFromQuery(plan)) { - execNonFromQuery(queryContext, plan, response, evalContext); + execNonFromQuery(queryContext, plan, response); } else { // it requires distributed execution. So, the query is forwarded to a query master. executeDistributedQuery(queryContext, session, plan, sql, jsonExpr, response); @@ -266,10 +266,11 @@ public void execSimpleQuery(QueryContext queryContext, Session session, String q response.setResultCode(ClientProtos.ResultCode.OK); } - public void execNonFromQuery(QueryContext queryContext, LogicalPlan plan, SubmitQueryResponse.Builder responseBuilder, - EvalContext evalContext) throws Exception { + public void execNonFromQuery(QueryContext queryContext, LogicalPlan plan, SubmitQueryResponse.Builder responseBuilder) + throws Exception { LogicalRootNode rootNode = plan.getRootBlock().getRoot(); + EvalContext evalContext = new EvalContext(); Target[] targets = plan.getRootBlock().getRawTargets(); if (targets == null) { throw new PlanningException("No targets"); @@ -307,7 +308,6 @@ public void execNonFromQuery(QueryContext queryContext, LogicalPlan plan, Submit } } - @VisibleForTesting public static void startScriptExecutors(QueryContext queryContext, EvalContext evalContext, Target[] targets) throws IOException { for (int i = 0; i < targets.length; i++) { @@ -323,7 +323,6 @@ public static void startScriptExecutors(QueryContext queryContext, EvalContext e } } - @VisibleForTesting public static void stopScriptExecutors(EvalContext evalContext) throws IOException { for (ScriptExecutor executor : evalContext.getAllScriptExecutors()) { executor.shutdown(); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 1fea8a9eda..9d6b1121a0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -133,7 +133,7 @@ public TajoConf getConf() { * @return * @throws PlanningException */ - private static Target[] getRawTargets(QueryContext context, EvalContext evalContext, String query, boolean condition) + private static Target[] getRawTargets(QueryContext context, String query, boolean condition) throws PlanningException, InvalidStatementException { List parsedResults = SimpleParser.parseScript(query); @@ -149,7 +149,7 @@ private static Target[] getRawTargets(QueryContext context, EvalContext evalCont } assertFalse(state.getErrorMessages().get(0), true); } - LogicalPlan plan = planner.createPlan(context, evalContext, expr, true); + LogicalPlan plan = planner.createPlan(context, expr, true); optimizer.optimize(context, plan); annotatedPlanVerifier.verify(context, state, plan); @@ -276,7 +276,7 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S EvalContext evalContext = new EvalContext(); try { - targets = getRawTargets(queryContext, evalContext, query, condition); + targets = getRawTargets(queryContext, query, condition); EvalCodeGenerator codegen = null; if (queryContext.getBool(SessionVars.CODEGEN)) { diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java index 6cad17b88a..0466a24480 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/TestEvalTreeUtil.java @@ -136,11 +136,11 @@ public static void tearDown() throws Exception { util.shutdownCatalogCluster(); } - public static Target [] getRawTargets(EvalContext evalContext, String query) { + public static Target [] getRawTargets(String query) { Expr expr = analyzer.parse(query); LogicalPlan plan = null; try { - plan = planner.createPlan(defaultContext, evalContext, expr); + plan = planner.createPlan(defaultContext, expr); } catch (PlanningException e) { e.printStackTrace(); } @@ -159,7 +159,7 @@ public static EvalNode getRootSelection(String query) throws PlanningException { } LogicalPlanner.PlanContext context = new LogicalPlanner.PlanContext(defaultContext, plan, plan.getRootBlock(), - new EvalTreeOptimizer(null), true); + new EvalTreeOptimizer(), true); Selection selection = plan.getRootBlock().getSingletonExpr(OpType.Filter); return planner.getExprAnnotator().createEvalNode(context, selection.getQual(), @@ -310,7 +310,7 @@ public final void testSimplify() throws PlanningException { assertTrue(7.0d == node.bind(null, null).eval(null).asFloat8()); Expr expr = analyzer.parse(QUERIES[1]); - LogicalPlan plan = planner.createPlan(defaultContext, new EvalContext(), expr, true); + LogicalPlan plan = planner.createPlan(defaultContext, expr, true); targets = plan.getRootBlock().getRawTargets(); Column col1 = new Column("default.people.score", TajoDataTypes.Type.INT4); Collection exprs = diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java index 3bf5db75e0..d1c1a150e5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/LogicalPlanner.java @@ -86,10 +86,9 @@ public static class PlanContext { EvalTreeOptimizer evalOptimizer; TimeZone timeZone; boolean debugOrUnitTests; - EvalContext evalContext; public PlanContext(OverridableConf context, LogicalPlan plan, QueryBlock block, EvalTreeOptimizer evalOptimizer, - EvalContext evalContext, boolean debugOrUnitTests) { + boolean debugOrUnitTests) { this.queryContext = context; this.plan = plan; this.queryBlock = block; @@ -101,7 +100,6 @@ public PlanContext(OverridableConf context, LogicalPlan plan, QueryBlock block, timeZone = TimeZone.getTimeZone(timezoneId); } - this.evalContext = evalContext; this.debugOrUnitTests = debugOrUnitTests; } @@ -111,15 +109,14 @@ public PlanContext(PlanContext context, QueryBlock block) { this.queryBlock = block; this.evalOptimizer = context.evalOptimizer; this.debugOrUnitTests = context.debugOrUnitTests; - this.evalContext = context.evalContext; } public QueryBlock getQueryBlock() { return queryBlock; } - public EvalContext getEvalContext() { - return evalContext; + public OverridableConf getQueryContext() { + return queryContext; } public String toString() { @@ -134,18 +131,18 @@ public String toString() { * @param expr A relational algebraic expression for a query. * @return A logical plan */ - public LogicalPlan createPlan(OverridableConf context, EvalContext evalContext, Expr expr) throws PlanningException { - return createPlan(context, evalContext, expr, false); + public LogicalPlan createPlan(OverridableConf context, Expr expr) throws PlanningException { + return createPlan(context, expr, false); } @VisibleForTesting - public LogicalPlan createPlan(OverridableConf queryContext, EvalContext evalContext, Expr expr, boolean debug) + public LogicalPlan createPlan(OverridableConf queryContext, Expr expr, boolean debug) throws PlanningException { LogicalPlan plan = new LogicalPlan(this); QueryBlock rootBlock = plan.newAndGetBlock(LogicalPlan.ROOT_BLOCK); - PlanContext context = new PlanContext(queryContext, plan, rootBlock, evalOptimizer, evalContext, debug); + PlanContext context = new PlanContext(queryContext, plan, rootBlock, evalOptimizer, debug); preprocessor.visit(context, new Stack(), expr); plan.resetGeneratedId(); LogicalNode topMostNode = this.visit(context, new Stack(), expr); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index df0b92d364..a166bccd8b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -18,11 +18,15 @@ package org.apache.tajo.plan.exprrewrite.rules; +import org.apache.tajo.datum.Datum; import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.exprrewrite.EvalTreeOptimizationRule; import org.apache.tajo.plan.annotator.Prioritized; import org.apache.tajo.plan.LogicalPlanner; +import org.apache.tajo.plan.function.python.PythonScriptExecutor; +import org.apache.tajo.plan.function.python.ScriptExecutor; +import java.io.IOException; import java.util.Stack; @Prioritized(priority = 10) @@ -51,7 +55,7 @@ public EvalNode visitBinaryEval(LogicalPlanner.PlanContext context, Stack Date: Sat, 11 Apr 2015 10:39:35 +0900 Subject: [PATCH 45/55] TAJO-1344_3 --- .../exprrewrite/rules/ConstantFolding.java | 28 +++++++++++-------- .../plan/function/python/ScriptExecutor.java | 18 ++++++++++++ 2 files changed, 35 insertions(+), 11 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index a166bccd8b..d2348a9059 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -91,18 +91,24 @@ public EvalNode visitFuncCall(LogicalPlanner.PlanContext context, FunctionEval e } if (constantOfAllDescendents && evalNode.getType() == EvalType.FUNCTION) { - ScriptExecutor executor = new PythonScriptExecutor(evalNode.getFuncDesc()); - try { - executor.start(context.getQueryContext()); - EvalContext evalContext = new EvalContext(); - evalContext.addScriptExecutor(evalNode, executor); - evalNode.bind(evalContext, null); - Datum funcRes = evalNode.eval(null); - executor.shutdown(); - return new ConstEval(funcRes); - } catch (IOException e) { - throw new RuntimeException(e); + if (evalNode.getFuncDesc().getInvocation().hasPython()) { + ScriptExecutor executor = new PythonScriptExecutor(evalNode.getFuncDesc()); + try { + executor.start(context.getQueryContext()); + EvalContext evalContext = new EvalContext(); + evalContext.addScriptExecutor(evalNode, executor); + evalNode.bind(evalContext, null); + Datum funcRes = evalNode.eval(null); + executor.shutdown(); + return new ConstEval(funcRes); + } catch (IOException e) { + throw new RuntimeException(e); + } + } else { + evalNode.bind(null, null); + return new ConstEval(evalNode.eval(null)); } + } else { return evalNode; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java index 3a8fe2ab0b..4b35bbbb09 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.tajo.plan.function.python; import org.apache.tajo.OverridableConf; From 33c2f25dab4683068e71200abd0e0ab078d9ed9f Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 12 Apr 2015 01:40:12 +0900 Subject: [PATCH 46/55] TAJO-1344_3 --- .../apache/tajo/catalog/store/MemStore.java | 1 - .../java/org/apache/tajo/conf/TajoConf.java | 1 - .../tajo/engine/function/FunctionLoader.java | 13 + .../java/org/apache/tajo/worker/Task.java | 6 - .../src/main/resources/python/controller.py | 26 +- .../engine/function/TestPythonFunctions.java | 2 +- .../plan/function/PythonFunctionInvoke.java | 21 -- .../function/python/PythonScriptEngine.java | 17 +- .../function/python/PythonScriptExecutor.java | 242 ++--------------- .../plan/function/python/ScriptExecutor.java | 1 - .../plan/function/stream/InputHandler.java | 10 +- .../plan/function/stream/OutputHandler.java | 29 +- .../function/stream/StreamingCommand.java | 247 ------------------ .../stream/StreamingUDFException.java | 75 ------ .../plan/function/stream/StreamingUtil.java | 21 +- 15 files changed, 69 insertions(+), 643 deletions(-) delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java index b0585048cc..821b00c9fb 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/MemStore.java @@ -30,7 +30,6 @@ import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.TableMeta; import org.apache.tajo.catalog.exception.*; -import org.apache.tajo.catalog.partition.PartitionDesc; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.catalog.proto.CatalogProtos.ColumnProto; import org.apache.tajo.catalog.proto.CatalogProtos.DatabaseProto; diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index d4b887d8e6..efb626cf85 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -293,7 +293,6 @@ public static enum ConfVars implements ConfigKey { GEOIP_DATA("tajo.function.geoip-database-location", ""), // Python UDF - PYTHON_CMD_ARGS_REMAINDERS("tajo.function.python.args-reminders", ""), PYTHON_CODE_DIR("tajo.function.python.code-dir", ""), ///////////////////////////////////////////////////////////////////////////////// diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index 506fc21c48..c259e7b89f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -54,6 +54,11 @@ public class FunctionLoader { private static Log LOG = LogFactory.getLog(FunctionLoader.class); public static final String PYTHON_FUNCTION_NAMESPACE = "python"; + /** + * Load built-in functions + * + * @return + */ public static Map load() { Map map = Maps.newHashMap(); @@ -78,6 +83,14 @@ public static Map load() { return map; } + /** + * Load functions that are optionally defined by users. + * + * @param conf + * @param functionMap + * @return + * @throws IOException + */ public static Map loadOptionalFunctions(TajoConf conf, Map functionMap) throws IOException { diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index dd188078f9..363bcd293a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -195,21 +195,15 @@ public void initPlan() throws IOException { } private void startScriptExecutors() throws IOException { - int cnt = 0; for (ScriptExecutor executor : context.getEvalContext().getAllScriptExecutors()) { executor.start(queryContext); - cnt++; } - LOG.info(cnt + " script executors are started."); } private void stopScriptExecutors() throws IOException { - int cnt = 0; for (ScriptExecutor executor : context.getEvalContext().getAllScriptExecutors()) { executor.shutdown(); - cnt++; } - LOG.info(cnt + " script executors are shutdowned."); } public void init() throws IOException { diff --git a/tajo-core/src/main/resources/python/controller.py b/tajo-core/src/main/resources/python/controller.py index 077864d455..d969b34b3d 100644 --- a/tajo-core/src/main/resources/python/controller.py +++ b/tajo-core/src/main/resources/python/controller.py @@ -95,9 +95,11 @@ def main(self, sys.path.append(cache_path) sys.path.append('.') - logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) - logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the " - "INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") + should_log = False + if should_log: + logging.basicConfig(filename=log_file_name, format="%(asctime)s %(levelname)s %(message)s", level=udf_logging.udf_log_level) + logging.info("To reduce the amount of information being logged only a small subset of rows are logged at the " + "INFO level. Call udf_logging.set_log_level_debug in tajo_util to see all rows being processed.") input_str = self.get_next_input() @@ -105,20 +107,11 @@ def main(self, func = __import__(module_name, globals(), locals(), [func_name], -1).__dict__[func_name] except: # These errors should always be caused by user code. - logging.info('write_user_exception1') write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-1) - # if udf_logging.udf_log_level != logging.DEBUG: - # #Only log output for illustrate after we get the flag to capture output. - # sys.stdout = open(os.devnull, 'w') - # else: - # sys.stdout = self.log_stream - - should_log = True log_message = logging.info if udf_logging.udf_log_level == logging.DEBUG: - should_log = True log_message = logging.debug while input_str != END_OF_STREAM: @@ -131,7 +124,6 @@ def main(self, log_message("Deserialized Input: %s" % (unicode(inputs))) except: # Capture errors where the user passes in bad data. - logging.info('write_user_exception2') write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-3) @@ -141,7 +133,6 @@ def main(self, log_message("UDF Output: %s" % (unicode(func_output))) except: # These errors should always be caused by user code. - logging.info('write_user_exception3') write_user_exception(module_name, self.stream_error, NUM_LINES_OFFSET_TRACE) self.close_controller(-2) @@ -154,7 +145,6 @@ def main(self, # This should only catch internal exceptions with the controller # and pig- not with user code. import traceback - logging.info('traceback') traceback.print_exc(file=self.stream_error) sys.exit(-3) @@ -181,11 +171,6 @@ def get_next_input(self): if input_str == '': return END_OF_STREAM - # if input_str == TURN_ON_OUTPUT_CAPTURING: - # logging.debug("Turned on Output Capturing") - # sys.stdout = log_stream - # return self.get_next_input() - if input_str == END_OF_STREAM: return input_str @@ -193,7 +178,6 @@ def get_next_input(self): def close_controller(self, exit_code): sys.stderr.close() - logging.info('last') self.stream_error.write("\n") self.stream_error.close() sys.stdout.close() diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java index 2435ad7978..47a0ad2599 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/function/TestPythonFunctions.java @@ -26,7 +26,7 @@ public class TestPythonFunctions extends ExprTestBase { @Test - public void test() throws IOException { + public void testFunctions() throws IOException { testSimpleEval("select return_one()", new String[]{"1"}); testSimpleEval("select helloworld()", new String[]{"Hello, World"}); testSimpleEval("select concat_py('1')", new String[]{"11"}); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index 266b7909e4..32c1dca8e7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -42,33 +42,12 @@ public PythonFunctionInvoke(FunctionDesc functionDesc) { @Override public void init(FunctionInvokeContext context) throws IOException { -// this.scriptExecutor = new PythonScriptExecutor(functionDesc); this.scriptExecutor = (PythonScriptExecutor) context.getScriptExecutor(); } @Override public Datum eval(Tuple tuple) { - // TODO: Currently, the script executor is started and stopped for every eval() call. - // TODO: Since it internally forks a child process which executes python functions, - // TODO: frequent calls of start/stop functions will incur a large overhead. - // TODO: To avoid this problem, PythonScriptExecutor should have the same life cycle with the TaskRunner. - // TODO: In addition, we should consider the resource management problem, too. -// if (scriptExecutor == null) { -// this.scriptExecutor = new PythonScriptExecutor(functionDesc); -// try { -// scriptExecutor.start(context); -// } catch (IOException e) { -// throw new RuntimeException(e); -// } -// } Datum res = scriptExecutor.eval(tuple); -// try { -// scriptExecutor.stop(); -// } catch (IOException e) { -// throw new RuntimeException(e); -// } catch (InterruptedException e) { -// throw new RuntimeException(e); -// } return res; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index d23837ac7d..87b31de449 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -18,8 +18,6 @@ package org.apache.tajo.plan.function.python; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.proto.CatalogProtos; @@ -30,7 +28,10 @@ import org.apache.tajo.function.PythonInvocationDesc; import org.apache.tajo.util.TUtil; -import java.io.*; +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; import java.net.URI; import java.nio.charset.Charset; import java.util.List; @@ -40,9 +41,17 @@ public class PythonScriptEngine extends TajoScriptEngine { public static final String FILE_EXTENSION = ".py"; - private static final Log LOG = LogFactory.getLog(PythonScriptEngine.class); + /** + * Register functions defined in a python script + * + * @param path path to the python script file + * @param namespace namespace where the functions will be defined + * @return set of function descriptions + * @throws IOException + */ public static Set registerFunctions(URI path, String namespace) throws IOException { + // TODO: we should support the namespace for python functions. Set functionDescs = TUtil.newHashSet(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java index f4056bb4ce..c923488faf 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java @@ -18,7 +18,6 @@ package org.apache.tajo.plan.function.python; -import com.google.common.base.Charsets; import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -36,9 +35,6 @@ import org.apache.tajo.storage.VTuple; import java.io.*; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.TimeUnit; /** * {@link PythonScriptExecutor} is a script executor for python functions. @@ -72,74 +68,49 @@ public class PythonScriptExecutor implements ScriptExecutor { private OverridableConf queryContext; private Process process; // Handle to the external execution of python functions - // all processes - private ProcessErrorThread stderrThread; // thread to get process stderr - private ProcessInputThread stdinThread; // thread to send input to process - private ProcessOutputThread stdoutThread; //thread to read output from process private InputHandler inputHandler; private OutputHandler outputHandler; - private BlockingQueue inputQueue; - private BlockingQueue outputQueue; - private DataOutputStream stdin; // stdin of the process private InputStream stdout; // stdout of the process private InputStream stderr; // stderr of the process - private static final Object ERROR_OUTPUT = new Object(); - private static final Object NULL_OBJECT = new Object(); - - private volatile StreamingUDFException outerrThreadsError; - -// private FunctionInvokeContext invokeContext = null; - private final FunctionSignature functionSignature; private final PythonInvocationDesc invocationDesc; private final Schema inSchema; private final Schema outSchema; - private final int [] projectionCols; + private final int [] projectionCols = new int[]{0}; private final CSVLineSerDe lineSerDe = new CSVLineSerDe(); - private final TableMeta pipeMeta; + private final TableMeta pipeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.TEXTFILE); - private boolean isStopped = false; + private static final Tuple EMPTY_INPUT = new VTuple(0); public PythonScriptExecutor(FunctionDesc functionDesc) { if (!functionDesc.getInvocation().hasPython()) { - throw new IllegalStateException("Function type must be python"); + throw new IllegalStateException("Function type must be 'python'"); } functionSignature = functionDesc.getSignature(); invocationDesc = functionDesc.getInvocation().getPython(); - // Compile input/output schema - // Note that temporal columns are used. TajoDataTypes.DataType[] paramTypes = functionSignature.getParamTypes(); inSchema = new Schema(); for (int i = 0; i < paramTypes.length; i++) { inSchema.addColumn(new Column("in_" + i, paramTypes[i])); } outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); - projectionCols = new int[]{0}; - pipeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.TEXTFILE); } -// public void start(FunctionInvokeContext context) throws IOException { public void start(OverridableConf queryContext) throws IOException { - isStopped = false; -// this.invokeContext = context; this.queryContext = queryContext; - this.inputQueue = new ArrayBlockingQueue(1); - this.outputQueue = new ArrayBlockingQueue(2); startUdfController(); createInputHandlers(); setStreams(); - startThreads(); - LOG.info("started"); + LOG.info("PythonScriptExecutor is started"); } public void shutdown() throws IOException { - isStopped = true; process.destroy(); if (stdin != null) { stdin.close(); @@ -152,24 +123,16 @@ public void shutdown() throws IOException { } inputHandler.close(process); outputHandler.close(); - LOG.info("shutdowned"); -// stdinThread.join(); -// stderrThread.join(); -// stdoutThread.join(); + LOG.info("PythonScriptExecutor is shutdowned"); } - private StreamingCommand startUdfController() throws IOException { - StreamingCommand sc = new StreamingCommand(buildCommand()); - ProcessBuilder processBuilder = StreamingUtil.createProcess(queryContext, sc); + private void startUdfController() throws IOException { + ProcessBuilder processBuilder = StreamingUtil.createProcess(buildCommand()); process = processBuilder.start(); - - Runtime.getRuntime().addShutdownHook(new ProcessKiller()); - - return sc; } /** - * Build a command to execute external process. + * Build a command to execute an external process. * @return * @throws IOException */ @@ -180,10 +143,8 @@ private String[] buildCommand() throws IOException { String standardOutputRootWriteLocation = ""; if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { LOG.warn("Currently, logging is not supported for the python controller."); - standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR); + standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR, DEFAULT_LOG_DIR); } -// standardOutputRootWriteLocation = "/home/jihoon/Projects/tajo/"; - standardOutputRootWriteLocation = "/Users/jihoonson/Projects/tajo/"; String controllerLogFileName, outFileName, errOutFileName; String funcName = invocationDesc.getName(); @@ -223,6 +184,11 @@ private void createInputHandlers() { this.outputHandler = new OutputHandler(deserializer); } + /** + * Get the standard input, output, and error streams of the external process + * + * @throws IOException + */ private void setStreams() throws IOException { stdout = new DataInputStream(new BufferedInputStream(process.getInputStream())); outputHandler.bindTo(stdout); @@ -233,17 +199,6 @@ private void setStreams() throws IOException { stderr = new DataInputStream(new BufferedInputStream(process.getErrorStream())); } - private void startThreads() { -// stdinThread = new ProcessInputThread(); -// stdinThread.start(); - -// stdoutThread = new ProcessOutputThread(); -// stdoutThread.start(); - -// stderrThread = new ProcessErrorThread(); -// stderrThread.start(); - } - /** * Find the path to the controller file for the streaming language. * @@ -280,185 +235,26 @@ private String getControllerPath() throws IOException { } public Datum eval(Tuple input) { - if (outputQueue == null) { - throw new RuntimeException("Process has already been shut down. No way to retrieve output for input: " + input); - } - try { if (input == null) { // When nothing is passed into the UDF the tuple // being sent is the full tuple for the relation. // We want it to be nothing (since that's what the user wrote). - input = new VTuple(0); + input = EMPTY_INPUT; } -// inputQueue.put(input); inputHandler.putNext(input); stdin.flush(); } catch (Exception e) { throw new RuntimeException("Failed adding input to inputQueue", e); } - Object o = null; + Datum result; try { - if (outputQueue != null) { -// o = outputQueue.take(); - o = outputHandler.getNext().get(0); - if (o == NULL_OBJECT) { - o = null; - } - } + result = outputHandler.getNext().get(0); } catch (Exception e) { throw new RuntimeException("Problem getting output", e); } - if (o == ERROR_OUTPUT) { - outputQueue = null; - if (outerrThreadsError == null) { - outerrThreadsError = new StreamingUDFException("python", "Problem with streaming udf. Can't recreate exception."); - } - throw new RuntimeException(outerrThreadsError); - } - - return (Datum) o; - } - - /** - * The thread which consumes input and feeds it to the the Process - */ - class ProcessInputThread extends Thread { - ProcessInputThread() { - setDaemon(true); - } - - public void run() { - try { - while (!isStopped) { - Tuple inputTuple = inputQueue.poll(10, TimeUnit.MILLISECONDS); - if (inputTuple != null) { - inputHandler.putNext(inputTuple); - } - try { - stdin.flush(); - } catch(Exception e) { - return; - } - } - } catch (InterruptedException e) { - - } catch (Exception e) { - LOG.error(e); - } - } - } - - private static final int WAIT_FOR_ERROR_LENGTH = 500; - private static final int MAX_WAIT_FOR_ERROR_ATTEMPTS = 5; - - /** - * The thread which consumes output from process - */ - class ProcessOutputThread extends Thread { - ProcessOutputThread() { - setDaemon(true); - } - - public void run() { - Object o; - try { - - o = outputHandler.getNext().get(0); - while (!isStopped && o != OutputHandler.END_OF_OUTPUT) { - if (o != null) { - outputQueue.put(o); - } - else { - outputQueue.put(NULL_OBJECT); - } - o = outputHandler.getNext().get(0); - } - } catch (IOException e) { - // EOF - } catch(Exception e) { - if (outputQueue != null) { - try { - // Give error thread a chance to check the standard error output - // for an exception message. - int attempt = 0; - while (stderrThread.isAlive() && attempt < MAX_WAIT_FOR_ERROR_ATTEMPTS) { - Thread.sleep(WAIT_FOR_ERROR_LENGTH); - attempt++; - } - // Only write this if no other error. Don't want to overwrite - // an error from the error thread. - if (outerrThreadsError == null) { - outerrThreadsError = new StreamingUDFException( - PYTHON_LANGUAGE, "Error deserializing output. Please check that the declared outputSchema for function " + - invocationDesc.getName() + " matches the data type being returned.", e); - } - // TODO: Currently, errors occurred before executing an input are ignored. - outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. - } catch(InterruptedException ie) { - LOG.error(ie); - } - } - } - } - } - - class ProcessErrorThread extends Thread { - public ProcessErrorThread() { - setDaemon(true); - } - - public void run() { - try { - Integer lineNumber = null; - StringBuffer error = new StringBuffer(); - String errInput; - BufferedReader reader = new BufferedReader( - new InputStreamReader(stderr, Charsets.UTF_8)); - while (!isStopped && ((errInput = reader.readLine()) != null)) { - // First line of error stream is usually the line number of error. - // If its not a number just treat it as first line of error message. - if (lineNumber == null) { - try { - lineNumber = Integer.valueOf(errInput); - } catch (NumberFormatException nfe) { - error.append(errInput + "\n"); - } - } else { - error.append(errInput + "\n"); - } - } - if (!isStopped) { - outerrThreadsError = new StreamingUDFException(PYTHON_LANGUAGE, error.toString(), lineNumber); - if (outputQueue != null) { - // TODO: Currently, errors occurred before executing an input are ignored. - outputQueue.put(ERROR_OUTPUT); // Need to wake main thread. - } - if (stderr != null) { - stderr.close(); - stderr = null; - } - } - } catch (IOException e) { - LOG.info("Process Ended", e); - } catch (Exception e) { - LOG.error("standard error problem", e); - } - } - } - - class ProcessKiller extends Thread { - public ProcessKiller() { - setDaemon(true); - } - public void run() { - try { - shutdown(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } + return result; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java index 4b35bbbb09..d87a26bb07 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java @@ -25,7 +25,6 @@ import java.io.IOException; public interface ScriptExecutor { -// void start(FunctionInvokeContext context) throws IOException; void start(OverridableConf queryContext) throws IOException; void shutdown() throws IOException; Datum eval(Tuple input); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java index f864f4dfe7..12e341a61a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -32,18 +32,12 @@ public class InputHandler { private final static byte[] END_OF_RECORD_DELIM = "|_\n".getBytes(); private final static byte[] END_OF_STREAM = ("C" + "\\x04" + "|_\n").getBytes(); - /** - * The serializer to be used to send data to the managed process. - * - * It is the responsibility of the concrete sub-classes to setup and - * manage the serializer. - */ - protected TextLineSerializer serializer; + private final TextLineSerializer serializer; private OutputStream out; // flag to mark if close() has already been called - protected boolean alreadyClosed = false; + private boolean alreadyClosed = false; public InputHandler(TextLineSerializer serializer) { this.serializer = serializer; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java index eeec77b20f..9e64288808 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java @@ -20,11 +20,6 @@ import com.google.common.base.Charsets; import io.netty.buffer.ByteBuf; -import org.apache.commons.codec.binary.Base64; -import org.apache.tajo.OverridableConf; -import org.apache.tajo.catalog.Schema; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.datum.NullDatum; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; @@ -37,12 +32,11 @@ */ public class OutputHandler { private static int DEFAULT_BUFFER = 64 * 1024; - public static final Object END_OF_OUTPUT = new Object(); - private static final byte[] DEFAULT_RECORD_DELIM = ",".getBytes(); + private final static byte[] END_OF_RECORD_DELIM = "|_\n".getBytes(); - protected TextLineDeserializer deserializer; + private final TextLineDeserializer deserializer; - protected ByteBufLineReader in = null; + private ByteBufLineReader in = null; private String currValue = null; @@ -50,14 +44,13 @@ public class OutputHandler { private final ByteBuf buf = BufferPool.directBuffer(DEFAULT_BUFFER); - //Both of these ignore the trailing \n. So if the - //default delimiter is "\n" recordDelimStr is "". + // Both of these ignore the trailing "\n". So if the default delimiter is "\n", recordDelimStr is "". private String recordDelimStr = null; private int recordDelimLength = 0; - private Tuple tuple = new VTuple(1); + private final Tuple tuple = new VTuple(1); // flag to mark if close() has already been called - protected boolean alreadyClosed = false; + private boolean alreadyClosed = false; public OutputHandler(TextLineDeserializer deserializer) { this.deserializer = deserializer; @@ -108,12 +101,12 @@ private boolean readValue() throws IOException { } while(!isEndOfRow()) { - //Need to add back the newline character we ate. + // Need to add back the newline character we ate. currValue += '\n'; byte[] lineBytes = readNextLine(); if (lineBytes == null) { - //We have no more input, so just break; + // We have no more input, so just break; break; } currValue += new String(lineBytes); @@ -138,7 +131,7 @@ private byte[] readNextLine() throws IOException { private boolean isEndOfRow() { if (recordDelimStr == null) { - byte[] recordDelimBa = getRecordDelimiter(); + byte[] recordDelimBa = END_OF_RECORD_DELIM; recordDelimLength = recordDelimBa.length - 1; //Ignore trailing \n recordDelimStr = new String(recordDelimBa, 0, recordDelimLength, Charsets.UTF_8); } @@ -148,10 +141,6 @@ private boolean isEndOfRow() { return currValue.contains(recordDelimStr); } - protected byte[] getRecordDelimiter() { - return DEFAULT_RECORD_DELIM; - } - /** * Close the OutputHandler. * @throws IOException diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java deleted file mode 100644 index 44d22c2128..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingCommand.java +++ /dev/null @@ -1,247 +0,0 @@ -/* - * Lisensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.stream; - -import java.io.File; -import java.io.IOException; -import java.io.Serializable; -import java.util.*; - -/** - * {@link StreamingCommand} represents the specification of an external - * command to be executed in a Pig Query. - * - * StreamingCommand encapsulates all relevant details of the - * command specified by the user either directly via the STREAM - * operator or indirectly via a DEFINE operator. It includes - * details such as input/output/error specifications and also files to be - * shipped to the cluster and files to be cached. - */ -public class StreamingCommand implements Serializable, Cloneable { - private static final long serialVersionUID = 1L; - - // External command to be executed and it's parsed components - String executable; - String[] argv; - - // Files to be shipped to the cluster in-order to be executed - List shipSpec = new LinkedList(); - - // Files to be shipped to the cluster in-order to be executed - List cacheSpec = new LinkedList(); - - /** - * Handle to communicate with the external process. - */ - public enum Handle {INPUT, OUTPUT} - - // Should the stderr of the process be persisted? - boolean persistStderr = false; - - // Directory where the process's stderr logs should be persisted. - String logDir; - - // Limit on the number of persisted log-files - int logFilesLimit = 100; - public static final int MAX_TASKS = 100; - - boolean shipFiles = true; - - /** - * Create a new StreamingCommand with the given command. - * - * @param argv parsed arguments of the command - */ - public StreamingCommand(String[] argv) { - this.argv = argv; - - // Assume that argv[0] is the executable - this.executable = this.argv[0]; - } - - /** - * Get the command to be executed. - * - * @return the command to be executed - */ - public String getExecutable() { - return executable; - } - - /** - * Set the executable for the StreamingCommand. - * - * @param executable the executable for the StreamingCommand - */ - public void setExecutable(String executable) { - this.executable = executable; - } - - /** - * Set the command line arguments for the StreamingCommand. - * - * @param argv the command line arguments for the - * StreamingCommand - */ - public void setCommandArgs(String[] argv) { - this.argv = argv; - } - - /** - * Get the parsed command arguments. - * - * @return the parsed command arguments as String[] - */ - public String[] getCommandArgs() { - return argv; - } - - /** - * Get the list of files which need to be shipped to the cluster. - * - * @return the list of files which need to be shipped to the cluster - */ - public List getShipSpecs() { - return shipSpec; - } - - /** - * Get the list of files which need to be cached on the execute nodes. - * - * @return the list of files which need to be cached on the execute nodes - */ - public List getCacheSpecs() { - return cacheSpec; - } - - /** - * Add a file to be shipped to the cluster. - * - * Users can use this to distribute executables and other necessary files - * to the clusters. - * - * @param path path of the file to be shipped to the cluster - */ - public void addPathToShip(String path) throws IOException { - // Validate - File file = new File(path); - if (!file.exists()) { - throw new IOException("Invalid ship specification: '" + path + - "' does not exist!"); - } else if (file.isDirectory()) { - throw new IOException("Invalid ship specification: '" + path + - "' is a directory and can't be shipped!"); - } - shipSpec.add(path); - } - - /** - * Should the stderr of the managed process be persisted? - * - * @return true if the stderr of the managed process should be - * persisted, false otherwise. - */ - public boolean getPersistStderr() { - return persistStderr; - } - - /** - * Specify if the stderr of the managed process should be persisted. - * - * @param persistStderr true if the stderr of the managed - * process should be persisted, else false - */ - public void setPersistStderr(boolean persistStderr) { - this.persistStderr = persistStderr; - } - - /** - * Get the directory where the log-files of the command are persisted. - * - * @return the directory where the log-files of the command are persisted - */ - public String getLogDir() { - return logDir; - } - - /** - * Set the directory where the log-files of the command are persisted. - * - * @param logDir the directory where the log-files of the command are persisted - */ - public void setLogDir(String logDir) { - this.logDir = logDir; - if (this.logDir.startsWith("/")) { - this.logDir = this.logDir.substring(1); - } - setPersistStderr(true); - } - - /** - * Get the maximum number of tasks whose stderr logs files are persisted. - * - * @return the maximum number of tasks whose stderr logs files are persisted - */ - public int getLogFilesLimit() { - return logFilesLimit; - } - - /** - * Set the maximum number of tasks whose stderr logs files are persisted. - * @param logFilesLimit the maximum number of tasks whose stderr logs files - * are persisted - */ - public void setLogFilesLimit(int logFilesLimit) { - this.logFilesLimit = Math.min(MAX_TASKS, logFilesLimit); - } - - /** - * Set whether files should be shipped or not. - * - * @param shipFiles true if files of this command should be - * shipped, false otherwise - */ - public void setShipFiles(boolean shipFiles) { - this.shipFiles = shipFiles; - } - - /** - * Get whether files for this command should be shipped or not. - * - * @return true if files of this command should be shipped, - * false otherwise - */ - public boolean getShipFiles() { - return shipFiles; - } - - public Object clone() { - try { - StreamingCommand clone = (StreamingCommand)super.clone(); - - clone.shipSpec = new ArrayList(shipSpec); - clone.cacheSpec = new ArrayList(cacheSpec); - - return clone; - } catch (CloneNotSupportedException cnse) { - // Shouldn't happen since we do implement Clonable - throw new InternalError(cnse.toString()); - } - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java deleted file mode 100644 index a9c974cb5c..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUDFException.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Lisensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.stream; - -public class StreamingUDFException extends Exception { - - private String message; - private String language; - private Integer lineNumber; - - public StreamingUDFException() { - } - - public StreamingUDFException(String message) { - this.message = message; - } - - public StreamingUDFException(String message, Integer lineNumber) { - this.message = message; - this.lineNumber = lineNumber; - } - - public StreamingUDFException(String language, String message, Throwable cause) { - super(cause); - this.language = language; - this.message = message + "\n" + cause.getMessage() + "\n"; - } - - public StreamingUDFException(String language, String message) { - this(language, message, (Integer) null); - } - - public StreamingUDFException(String language, String message, Integer lineNumber) { - this.language = language; - this.message = message; - this.lineNumber = lineNumber; - } - - public String getLanguage() { - return language; - } - - public Integer getLineNumber() { - return lineNumber; - } - - @Override - public String getMessage() { - return this.message; - } - - @Override - public String toString() { - String s = getClass().getName(); - String message = getMessage(); - String lineNumber = this.getLineNumber() == null ? "" : "" + this.getLineNumber(); - return (message != null) ? (s + ": " + "LINE " + lineNumber + ": " + message) : s; - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUtil.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUtil.java index 607353723c..9bb60dd10a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUtil.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/StreamingUtil.java @@ -18,32 +18,26 @@ package org.apache.tajo.plan.function.stream; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.tajo.OverridableConf; +import org.apache.tajo.util.TUtil; import java.io.File; -import java.util.ArrayList; import java.util.List; import java.util.Map; public class StreamingUtil { - private static Log LOG = LogFactory.getLog(StreamingUtil.class); - private static final String BASH = "bash"; private static final String PATH = "PATH"; /** * Create an external process for StreamingCommand command. * - * @param command + * @param argv process arguments * @return */ - public static ProcessBuilder createProcess(OverridableConf queryContext, StreamingCommand command) { + public static ProcessBuilder createProcess(String[] argv) { // Set the actual command to run with 'bash -c exec ...' - List cmdArgs = new ArrayList(); - String[] argv = command.getCommandArgs(); + List cmdArgs = TUtil.newList(); StringBuffer argBuffer = new StringBuffer(); for (String arg : argv) { @@ -68,17 +62,16 @@ public static ProcessBuilder createProcess(OverridableConf queryContext, Streami // Start the external process ProcessBuilder processBuilder = new ProcessBuilder(cmdArgs .toArray(new String[cmdArgs.size()])); - setupEnvironment(queryContext, processBuilder); + setupEnvironment(processBuilder); return processBuilder; } /** * Set up the run-time environment of the managed process. * - * @param pb - * {@link ProcessBuilder} used to exec the process + * @param pb {@link ProcessBuilder} used to exec the process */ - private static void setupEnvironment(OverridableConf queryContext, ProcessBuilder pb) { + private static void setupEnvironment(ProcessBuilder pb) { String separator = ":"; Map env = pb.environment(); From 658def734abb0d5bf00f4b156f5f1fc4af87d4ed Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 12 Apr 2015 12:02:57 +0900 Subject: [PATCH 47/55] TAJO-1344_3 --- .../tajo/master/exec/QueryExecutor.java | 11 +- .../java/org/apache/tajo/worker/Task.java | 10 +- .../apache/tajo/plan/expr/EvalContext.java | 20 +- .../tajo/plan/expr/GeneralFunctionEval.java | 4 +- .../exprrewrite/rules/ConstantFolding.java | 8 +- .../plan/function/FunctionInvokeContext.java | 15 +- .../plan/function/PythonFunctionInvoke.java | 8 +- .../function/python/PythonScriptEngine.java | 234 +++++++++++++++- .../function/python/PythonScriptExecutor.java | 260 ------------------ .../plan/function/python/ScriptExecutor.java | 31 --- .../function/python/TajoScriptEngine.java | 31 ++- .../plan/serder/EvalNodeDeserializer.java | 4 +- 12 files changed, 294 insertions(+), 342 deletions(-) delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java delete mode 100644 tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index 247a11f06f..d81660b070 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -18,7 +18,6 @@ package org.apache.tajo.master.exec; -import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -50,8 +49,8 @@ import org.apache.tajo.master.exec.prehook.InsertIntoHook; import org.apache.tajo.plan.expr.EvalContext; import org.apache.tajo.plan.expr.GeneralFunctionEval; -import org.apache.tajo.plan.function.python.PythonScriptExecutor; -import org.apache.tajo.plan.function.python.ScriptExecutor; +import org.apache.tajo.plan.function.python.PythonScriptEngine; +import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.plan.rewrite.LogicalPlanRewriteRule; import org.apache.tajo.querymaster.*; import org.apache.tajo.session.Session; @@ -315,8 +314,8 @@ public static void startScriptExecutors(QueryContext queryContext, EvalContext e if (eval instanceof GeneralFunctionEval) { GeneralFunctionEval functionEval = (GeneralFunctionEval) eval; if (functionEval.getFuncDesc().getInvocation().hasPython()) { - PythonScriptExecutor scriptExecutor = new PythonScriptExecutor(functionEval.getFuncDesc()); - evalContext.addScriptExecutor(eval, scriptExecutor); + TajoScriptEngine scriptExecutor = new PythonScriptEngine(functionEval.getFuncDesc()); + evalContext.addScriptEngine(eval, scriptExecutor); scriptExecutor.start(queryContext); } } @@ -324,7 +323,7 @@ public static void startScriptExecutors(QueryContext queryContext, EvalContext e } public static void stopScriptExecutors(EvalContext evalContext) throws IOException { - for (ScriptExecutor executor : evalContext.getAllScriptExecutors()) { + for (TajoScriptEngine executor : evalContext.getAllScriptEngines()) { executor.shutdown(); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index 363bcd293a..d432520108 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -46,8 +46,8 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.*; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.function.python.PythonScriptExecutor; -import org.apache.tajo.plan.function.python.ScriptExecutor; +import org.apache.tajo.plan.function.python.PythonScriptEngine; +import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.serder.LogicalNodeDeserializer; import org.apache.tajo.plan.util.PlannerUtil; @@ -102,7 +102,7 @@ public class Task { private Schema finalSchema = null; private TupleComparator sortComp = null; - private PythonScriptExecutor pythonExecutor; + private PythonScriptEngine pythonEngine; public Task(String taskRunnerId, Path baseDir, @@ -195,13 +195,13 @@ public void initPlan() throws IOException { } private void startScriptExecutors() throws IOException { - for (ScriptExecutor executor : context.getEvalContext().getAllScriptExecutors()) { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { executor.start(queryContext); } } private void stopScriptExecutors() throws IOException { - for (ScriptExecutor executor : context.getEvalContext().getAllScriptExecutors()) { + for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { executor.shutdown(); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java index 24f493fae0..6a30e772ce 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java @@ -18,28 +18,28 @@ package org.apache.tajo.plan.expr; -import org.apache.tajo.plan.function.python.ScriptExecutor; +import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.util.TUtil; import java.util.Collection; import java.util.Map; public class EvalContext { - private final Map scriptExecutorMap = TUtil.newHashMap(); + private final Map scriptEngineMap = TUtil.newHashMap(); - public void addScriptExecutor(EvalNode evalNode, ScriptExecutor scriptExecutor) { - this.scriptExecutorMap.put(evalNode, scriptExecutor); + public void addScriptEngine(EvalNode evalNode, TajoScriptEngine scriptExecutor) { + this.scriptEngineMap.put(evalNode, scriptExecutor); } - public boolean hasScriptExecutor(EvalNode evalNode) { - return this.scriptExecutorMap.containsKey(evalNode); + public boolean hasScriptEngine(EvalNode evalNode) { + return this.scriptEngineMap.containsKey(evalNode); } - public ScriptExecutor getScriptExecutor(EvalNode evalNode) { - return this.scriptExecutorMap.get(evalNode); + public TajoScriptEngine getScriptEngine(EvalNode evalNode) { + return this.scriptEngineMap.get(evalNode); } - public Collection getAllScriptExecutors() { - return this.scriptExecutorMap.values(); + public Collection getAllScriptEngines() { + return this.scriptEngineMap.values(); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 2e15534413..30fbe910f5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -44,8 +44,8 @@ public EvalNode bind(EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); try { this.funcInvoke = FunctionInvoke.newInstance(funcDesc); - if (evalContext != null && evalContext.hasScriptExecutor(this)) { - this.invokeContext.setScriptExecutor(evalContext.getScriptExecutor(this)); + if (evalContext != null && evalContext.hasScriptEngine(this)) { + this.invokeContext.setScriptEngine(evalContext.getScriptEngine(this)); } this.funcInvoke.init(invokeContext); } catch (IOException e) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index d2348a9059..942c3b47cd 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -23,8 +23,8 @@ import org.apache.tajo.plan.exprrewrite.EvalTreeOptimizationRule; import org.apache.tajo.plan.annotator.Prioritized; import org.apache.tajo.plan.LogicalPlanner; -import org.apache.tajo.plan.function.python.PythonScriptExecutor; -import org.apache.tajo.plan.function.python.ScriptExecutor; +import org.apache.tajo.plan.function.python.PythonScriptEngine; +import org.apache.tajo.plan.function.python.TajoScriptEngine; import java.io.IOException; import java.util.Stack; @@ -92,11 +92,11 @@ public EvalNode visitFuncCall(LogicalPlanner.PlanContext context, FunctionEval e if (constantOfAllDescendents && evalNode.getType() == EvalType.FUNCTION) { if (evalNode.getFuncDesc().getInvocation().hasPython()) { - ScriptExecutor executor = new PythonScriptExecutor(evalNode.getFuncDesc()); + TajoScriptEngine executor = new PythonScriptEngine(evalNode.getFuncDesc()); try { executor.start(context.getQueryContext()); EvalContext evalContext = new EvalContext(); - evalContext.addScriptExecutor(evalNode, executor); + evalContext.addScriptEngine(evalNode, executor); evalNode.bind(evalContext, null); Datum funcRes = evalNode.eval(null); executor.shutdown(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java index 42c7da5045..b9380723ac 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvokeContext.java @@ -20,9 +20,8 @@ import com.google.common.base.Objects; import org.apache.tajo.OverridableConf; -import org.apache.tajo.annotation.Nullable; import org.apache.tajo.plan.expr.FunctionEval; -import org.apache.tajo.plan.function.python.ScriptExecutor; +import org.apache.tajo.plan.function.python.TajoScriptEngine; import java.util.Arrays; @@ -32,7 +31,7 @@ public class FunctionInvokeContext { private final OverridableConf queryContext; private final FunctionEval.ParamType[] paramTypes; - private ScriptExecutor scriptExecutor; + private TajoScriptEngine scriptEngine; public FunctionInvokeContext(OverridableConf queryContext, FunctionEval.ParamType[] paramTypes) { this.queryContext = queryContext; @@ -47,16 +46,16 @@ public FunctionEval.ParamType[] getParamTypes() { return paramTypes; } - public void setScriptExecutor(ScriptExecutor scriptExecutor) { - this.scriptExecutor = scriptExecutor; + public void setScriptEngine(TajoScriptEngine scriptEngine) { + this.scriptEngine = scriptEngine; } public boolean hasScriptExecutor() { - return scriptExecutor != null; + return scriptEngine != null; } - public ScriptExecutor getScriptExecutor() { - return scriptExecutor; + public TajoScriptEngine getScriptEngine() { + return scriptEngine; } @Override diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index 32c1dca8e7..e6ab229c83 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -20,7 +20,7 @@ import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.datum.Datum; -import org.apache.tajo.plan.function.python.PythonScriptExecutor; +import org.apache.tajo.plan.function.python.PythonScriptEngine; import org.apache.tajo.storage.Tuple; import java.io.IOException; @@ -30,7 +30,7 @@ */ public class PythonFunctionInvoke extends FunctionInvoke implements Cloneable { - private PythonScriptExecutor scriptExecutor; + private PythonScriptEngine scriptEngine; public PythonFunctionInvoke() { @@ -42,12 +42,12 @@ public PythonFunctionInvoke(FunctionDesc functionDesc) { @Override public void init(FunctionInvokeContext context) throws IOException { - this.scriptExecutor = (PythonScriptExecutor) context.getScriptExecutor(); + this.scriptEngine = (PythonScriptEngine) context.getScriptEngine(); } @Override public Datum eval(Tuple tuple) { - Datum res = scriptExecutor.eval(tuple); + Datum res = scriptEngine.eval(tuple); return res; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index 87b31de449..4faf7a1b68 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -18,20 +18,26 @@ package org.apache.tajo.plan.function.python; -import org.apache.tajo.catalog.CatalogUtil; -import org.apache.tajo.catalog.FunctionDesc; +import org.apache.commons.io.FileUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.QueryVars; +import org.apache.tajo.catalog.*; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.datum.Datum; import org.apache.tajo.function.FunctionInvocation; import org.apache.tajo.function.FunctionSignature; import org.apache.tajo.function.FunctionSupplement; import org.apache.tajo.function.PythonInvocationDesc; +import org.apache.tajo.plan.function.stream.*; +import org.apache.tajo.storage.Tuple; +import org.apache.tajo.storage.VTuple; import org.apache.tajo.util.TUtil; -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; +import java.io.*; import java.net.URI; import java.nio.charset.Charset; import java.util.List; @@ -40,6 +46,8 @@ public class PythonScriptEngine extends TajoScriptEngine { + private static final Log LOG = LogFactory.getLog(PythonScriptEngine.class); + public static final String FILE_EXTENSION = ".py"; /** @@ -139,4 +147,218 @@ private static List getFunctions(InputStream is) throws IOException { in.close(); return functions; } + + + private static final String PYTHON_LANGUAGE = "python"; + private static final String PYTHON_ROOT_PATH = "/python"; + private static final String TAJO_UTIL_NAME = "tajo_util.py"; + private static final String CONTROLLER_NAME = "controller.py"; + private static final String PYTHON_CONTROLLER_JAR_PATH = PYTHON_ROOT_PATH + File.separator + CONTROLLER_NAME; // Relative to root of tajo jar. + private static final String PYTHON_TAJO_UTIL_PATH = PYTHON_ROOT_PATH + File.separator + TAJO_UTIL_NAME; // Relative to root of tajo jar. + private static final String DEFAULT_LOG_DIR = "/tmp/tajo-" + System.getProperty("user.name") + "/python"; + + // Indexes for arguments being passed to external process + private static final int UDF_LANGUAGE = 0; + private static final int PATH_TO_CONTROLLER_FILE = 1; + private static final int UDF_FILE_NAME = 2; // Name of file where UDF function is defined + private static final int UDF_FILE_PATH = 3; // Path to directory containing file where UDF function is defined + private static final int UDF_NAME = 4; // Name of UDF function being called. + private static final int PATH_TO_FILE_CACHE = 5; // Directory where required files (like tajo_util) are cached on cluster nodes. + private static final int STD_OUT_OUTPUT_PATH = 6; // File for output from when user writes to standard output. + private static final int STD_ERR_OUTPUT_PATH = 7; // File for output from when user writes to standard error. + private static final int CONTROLLER_LOG_FILE_PATH = 8; // Controller log file logs progress through the controller script not user code. + private static final int OUT_SCHEMA = 9; // the schema of the output column + + private OverridableConf queryContext; + + private Process process; // Handle to the external execution of python functions + + private InputHandler inputHandler; + private OutputHandler outputHandler; + + private DataOutputStream stdin; // stdin of the process + private InputStream stdout; // stdout of the process + private InputStream stderr; // stderr of the process + + private final FunctionSignature functionSignature; + private final PythonInvocationDesc invocationDesc; + private final Schema inSchema; + private final Schema outSchema; + private final int [] projectionCols = new int[]{0}; + + private final CSVLineSerDe lineSerDe = new CSVLineSerDe(); + private final TableMeta pipeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.TEXTFILE); + + private static final Tuple EMPTY_INPUT = new VTuple(0); + + public PythonScriptEngine(FunctionDesc functionDesc) { + if (!functionDesc.getInvocation().hasPython()) { + throw new IllegalStateException("Function type must be 'python'"); + } + functionSignature = functionDesc.getSignature(); + invocationDesc = functionDesc.getInvocation().getPython(); + + TajoDataTypes.DataType[] paramTypes = functionSignature.getParamTypes(); + inSchema = new Schema(); + for (int i = 0; i < paramTypes.length; i++) { + inSchema.addColumn(new Column("in_" + i, paramTypes[i])); + } + outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); + } + + public void start(OverridableConf queryContext) throws IOException { + this.queryContext = queryContext; + startUdfController(); + createInputHandlers(); + setStreams(); + LOG.info("PythonScriptExecutor is started"); + } + + public void shutdown() throws IOException { + process.destroy(); + if (stdin != null) { + stdin.close(); + } + if (stdout != null) { + stdout.close(); + } + if (stderr != null) { + stderr.close(); + } + inputHandler.close(process); + outputHandler.close(); + LOG.info("PythonScriptExecutor is shutdowned"); + } + + private void startUdfController() throws IOException { + ProcessBuilder processBuilder = StreamingUtil.createProcess(buildCommand()); + process = processBuilder.start(); + } + + /** + * Build a command to execute an external process. + * @return + * @throws IOException + */ + private String[] buildCommand() throws IOException { + String[] command = new String[10]; + + // TODO: support controller logging + String standardOutputRootWriteLocation = ""; + if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { + LOG.warn("Currently, logging is not supported for the python controller."); + standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR, DEFAULT_LOG_DIR); + } + String controllerLogFileName, outFileName, errOutFileName; + + String funcName = invocationDesc.getName(); + String filePath = invocationDesc.getPath(); + + controllerLogFileName = standardOutputRootWriteLocation + funcName + "_controller.log"; + outFileName = standardOutputRootWriteLocation + funcName + ".out"; + errOutFileName = standardOutputRootWriteLocation + funcName + ".err"; + + command[UDF_LANGUAGE] = PYTHON_LANGUAGE; + command[PATH_TO_CONTROLLER_FILE] = getControllerPath(); + int lastSeparator = filePath.lastIndexOf(File.separator) + 1; + String fileName = filePath.substring(lastSeparator); + fileName = fileName.endsWith(FILE_EXTENSION) ? fileName.substring(0, fileName.length()-3) : fileName; + command[UDF_FILE_NAME] = fileName; + command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); + command[UDF_NAME] = funcName; + if (!queryContext.containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { + throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); + } + String fileCachePath = queryContext.get(QueryVars.PYTHON_SCRIPT_CODE_DIR); + command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; + command[STD_OUT_OUTPUT_PATH] = outFileName; + command[STD_ERR_OUTPUT_PATH] = errOutFileName; + command[CONTROLLER_LOG_FILE_PATH] = controllerLogFileName; + command[OUT_SCHEMA] = outSchema.getColumn(0).getDataType().getType().name().toLowerCase(); + + return command; + } + + private void createInputHandlers() { + TextLineSerializer serializer = lineSerDe.createSerializer(inSchema, pipeMeta); + serializer.init(); + this.inputHandler = new InputHandler(serializer); + TextLineDeserializer deserializer = lineSerDe.createDeserializer(outSchema, pipeMeta, projectionCols); + deserializer.init(); + this.outputHandler = new OutputHandler(deserializer); + } + + /** + * Get the standard input, output, and error streams of the external process + * + * @throws IOException + */ + private void setStreams() throws IOException { + stdout = new DataInputStream(new BufferedInputStream(process.getInputStream())); + outputHandler.bindTo(stdout); + + stdin = new DataOutputStream(new BufferedOutputStream(process.getOutputStream())); + inputHandler.bindTo(stdin); + + stderr = new DataInputStream(new BufferedInputStream(process.getErrorStream())); + } + + /** + * Find the path to the controller file for the streaming language. + * + * First check path to job jar and if the file is not found (like in the + * case of running hadoop in standalone mode) write the necessary files + * to temporary files and return that path. + * + * @return + * @throws IOException + */ + private String getControllerPath() throws IOException { + String controllerPath = PYTHON_CONTROLLER_JAR_PATH; + File controller = new File(PYTHON_CONTROLLER_JAR_PATH); + if (!controller.exists()) { + File controllerFile = File.createTempFile("controller", FILE_EXTENSION); + InputStream pythonControllerStream = this.getClass().getResourceAsStream(PYTHON_CONTROLLER_JAR_PATH); + try { + FileUtils.copyInputStreamToFile(pythonControllerStream, controllerFile); + } finally { + pythonControllerStream.close(); + } + controllerFile.deleteOnExit(); + File tajoUtilFile = new File(controllerFile.getParent() + File.separator + TAJO_UTIL_NAME); + tajoUtilFile.deleteOnExit(); + InputStream pythonUtilStream = this.getClass().getResourceAsStream(PYTHON_TAJO_UTIL_PATH); + try { + FileUtils.copyInputStreamToFile(pythonUtilStream, tajoUtilFile); + } finally { + pythonUtilStream.close(); + } + controllerPath = controllerFile.getAbsolutePath(); + } + return controllerPath; + } + + public Datum eval(Tuple input) { + try { + if (input == null) { + // When nothing is passed into the UDF the tuple + // being sent is the full tuple for the relation. + // We want it to be nothing (since that's what the user wrote). + input = EMPTY_INPUT; + } + + inputHandler.putNext(input); + stdin.flush(); + } catch (Exception e) { + throw new RuntimeException("Failed adding input to inputQueue", e); + } + Datum result; + try { + result = outputHandler.getNext().get(0); + } catch (Exception e) { + throw new RuntimeException("Problem getting output", e); + } + + return result; + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java deleted file mode 100644 index c923488faf..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptExecutor.java +++ /dev/null @@ -1,260 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.python; - -import org.apache.commons.io.FileUtils; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.tajo.OverridableConf; -import org.apache.tajo.QueryVars; -import org.apache.tajo.catalog.*; -import org.apache.tajo.catalog.proto.CatalogProtos; -import org.apache.tajo.common.TajoDataTypes; -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.function.FunctionSignature; -import org.apache.tajo.function.PythonInvocationDesc; -import org.apache.tajo.plan.function.stream.*; -import org.apache.tajo.storage.Tuple; -import org.apache.tajo.storage.VTuple; - -import java.io.*; - -/** - * {@link PythonScriptExecutor} is a script executor for python functions. - * It internally creates a child process which is responsible for executing python codes. - * Given an input tuple, it sends the tuple to the child process, and then receives a result from that. - */ -public class PythonScriptExecutor implements ScriptExecutor { - - private static final Log LOG = LogFactory.getLog(PythonScriptExecutor.class); - - private static final String PYTHON_LANGUAGE = "python"; - private static final String PYTHON_ROOT_PATH = "/python"; - private static final String TAJO_UTIL_NAME = "tajo_util.py"; - private static final String CONTROLLER_NAME = "controller.py"; - private static final String PYTHON_CONTROLLER_JAR_PATH = PYTHON_ROOT_PATH + File.separator + CONTROLLER_NAME; // Relative to root of tajo jar. - private static final String PYTHON_TAJO_UTIL_PATH = PYTHON_ROOT_PATH + File.separator + TAJO_UTIL_NAME; // Relative to root of tajo jar. - private static final String DEFAULT_LOG_DIR = "/tmp/tajo-" + System.getProperty("user.name") + "/python"; - - // Indexes for arguments being passed to external process - private static final int UDF_LANGUAGE = 0; - private static final int PATH_TO_CONTROLLER_FILE = 1; - private static final int UDF_FILE_NAME = 2; // Name of file where UDF function is defined - private static final int UDF_FILE_PATH = 3; // Path to directory containing file where UDF function is defined - private static final int UDF_NAME = 4; // Name of UDF function being called. - private static final int PATH_TO_FILE_CACHE = 5; // Directory where required files (like tajo_util) are cached on cluster nodes. - private static final int STD_OUT_OUTPUT_PATH = 6; // File for output from when user writes to standard output. - private static final int STD_ERR_OUTPUT_PATH = 7; // File for output from when user writes to standard error. - private static final int CONTROLLER_LOG_FILE_PATH = 8; // Controller log file logs progress through the controller script not user code. - private static final int OUT_SCHEMA = 9; // the schema of the output column - - private OverridableConf queryContext; - - private Process process; // Handle to the external execution of python functions - - private InputHandler inputHandler; - private OutputHandler outputHandler; - - private DataOutputStream stdin; // stdin of the process - private InputStream stdout; // stdout of the process - private InputStream stderr; // stderr of the process - - private final FunctionSignature functionSignature; - private final PythonInvocationDesc invocationDesc; - private final Schema inSchema; - private final Schema outSchema; - private final int [] projectionCols = new int[]{0}; - - private final CSVLineSerDe lineSerDe = new CSVLineSerDe(); - private final TableMeta pipeMeta = CatalogUtil.newTableMeta(CatalogProtos.StoreType.TEXTFILE); - - private static final Tuple EMPTY_INPUT = new VTuple(0); - - public PythonScriptExecutor(FunctionDesc functionDesc) { - if (!functionDesc.getInvocation().hasPython()) { - throw new IllegalStateException("Function type must be 'python'"); - } - functionSignature = functionDesc.getSignature(); - invocationDesc = functionDesc.getInvocation().getPython(); - - TajoDataTypes.DataType[] paramTypes = functionSignature.getParamTypes(); - inSchema = new Schema(); - for (int i = 0; i < paramTypes.length; i++) { - inSchema.addColumn(new Column("in_" + i, paramTypes[i])); - } - outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); - } - - public void start(OverridableConf queryContext) throws IOException { - this.queryContext = queryContext; - startUdfController(); - createInputHandlers(); - setStreams(); - LOG.info("PythonScriptExecutor is started"); - } - - public void shutdown() throws IOException { - process.destroy(); - if (stdin != null) { - stdin.close(); - } - if (stdout != null) { - stdout.close(); - } - if (stderr != null) { - stderr.close(); - } - inputHandler.close(process); - outputHandler.close(); - LOG.info("PythonScriptExecutor is shutdowned"); - } - - private void startUdfController() throws IOException { - ProcessBuilder processBuilder = StreamingUtil.createProcess(buildCommand()); - process = processBuilder.start(); - } - - /** - * Build a command to execute an external process. - * @return - * @throws IOException - */ - private String[] buildCommand() throws IOException { - String[] command = new String[10]; - - // TODO: support controller logging - String standardOutputRootWriteLocation = ""; - if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { - LOG.warn("Currently, logging is not supported for the python controller."); - standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR, DEFAULT_LOG_DIR); - } - String controllerLogFileName, outFileName, errOutFileName; - - String funcName = invocationDesc.getName(); - String filePath = invocationDesc.getPath(); - - controllerLogFileName = standardOutputRootWriteLocation + funcName + "_controller.log"; - outFileName = standardOutputRootWriteLocation + funcName + ".out"; - errOutFileName = standardOutputRootWriteLocation + funcName + ".err"; - - command[UDF_LANGUAGE] = PYTHON_LANGUAGE; - command[PATH_TO_CONTROLLER_FILE] = getControllerPath(); - int lastSeparator = filePath.lastIndexOf(File.separator) + 1; - String fileName = filePath.substring(lastSeparator); - fileName = fileName.endsWith(".py") ? fileName.substring(0, fileName.length()-3) : fileName; - command[UDF_FILE_NAME] = fileName; - command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); - command[UDF_NAME] = funcName; - if (!queryContext.containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { - throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); - } - String fileCachePath = queryContext.get(QueryVars.PYTHON_SCRIPT_CODE_DIR); - command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; - command[STD_OUT_OUTPUT_PATH] = outFileName; - command[STD_ERR_OUTPUT_PATH] = errOutFileName; - command[CONTROLLER_LOG_FILE_PATH] = controllerLogFileName; - command[OUT_SCHEMA] = outSchema.getColumn(0).getDataType().getType().name().toLowerCase(); - - return command; - } - - private void createInputHandlers() { - TextLineSerializer serializer = lineSerDe.createSerializer(inSchema, pipeMeta); - serializer.init(); - this.inputHandler = new InputHandler(serializer); - TextLineDeserializer deserializer = lineSerDe.createDeserializer(outSchema, pipeMeta, projectionCols); - deserializer.init(); - this.outputHandler = new OutputHandler(deserializer); - } - - /** - * Get the standard input, output, and error streams of the external process - * - * @throws IOException - */ - private void setStreams() throws IOException { - stdout = new DataInputStream(new BufferedInputStream(process.getInputStream())); - outputHandler.bindTo(stdout); - - stdin = new DataOutputStream(new BufferedOutputStream(process.getOutputStream())); - inputHandler.bindTo(stdin); - - stderr = new DataInputStream(new BufferedInputStream(process.getErrorStream())); - } - - /** - * Find the path to the controller file for the streaming language. - * - * First check path to job jar and if the file is not found (like in the - * case of running hadoop in standalone mode) write the necessary files - * to temporary files and return that path. - * - * @return - * @throws IOException - */ - private String getControllerPath() throws IOException { - String controllerPath = PYTHON_CONTROLLER_JAR_PATH; - File controller = new File(PYTHON_CONTROLLER_JAR_PATH); - if (!controller.exists()) { - File controllerFile = File.createTempFile("controller", ".py"); - InputStream pythonControllerStream = this.getClass().getResourceAsStream(PYTHON_CONTROLLER_JAR_PATH); - try { - FileUtils.copyInputStreamToFile(pythonControllerStream, controllerFile); - } finally { - pythonControllerStream.close(); - } - controllerFile.deleteOnExit(); - File tajoUtilFile = new File(controllerFile.getParent() + File.separator + TAJO_UTIL_NAME); - tajoUtilFile.deleteOnExit(); - InputStream pythonUtilStream = this.getClass().getResourceAsStream(PYTHON_TAJO_UTIL_PATH); - try { - FileUtils.copyInputStreamToFile(pythonUtilStream, tajoUtilFile); - } finally { - pythonUtilStream.close(); - } - controllerPath = controllerFile.getAbsolutePath(); - } - return controllerPath; - } - - public Datum eval(Tuple input) { - try { - if (input == null) { - // When nothing is passed into the UDF the tuple - // being sent is the full tuple for the relation. - // We want it to be nothing (since that's what the user wrote). - input = EMPTY_INPUT; - } - - inputHandler.putNext(input); - stdin.flush(); - } catch (Exception e) { - throw new RuntimeException("Failed adding input to inputQueue", e); - } - Datum result; - try { - result = outputHandler.getNext().get(0); - } catch (Exception e) { - throw new RuntimeException("Problem getting output", e); - } - - return result; - } -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java deleted file mode 100644 index d87a26bb07..0000000000 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/ScriptExecutor.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.plan.function.python; - -import org.apache.tajo.OverridableConf; -import org.apache.tajo.datum.Datum; -import org.apache.tajo.storage.Tuple; - -import java.io.IOException; - -public interface ScriptExecutor { - void start(OverridableConf queryContext) throws IOException; - void shutdown() throws IOException; - Datum eval(Tuple input); -} diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index b915318480..1d9e9c926e 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -18,10 +18,11 @@ package org.apache.tajo.plan.function.python; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileNotFoundException; -import java.io.InputStream; +import org.apache.tajo.OverridableConf; +import org.apache.tajo.datum.Datum; +import org.apache.tajo.storage.Tuple; + +import java.io.*; import java.net.URI; public abstract class TajoScriptEngine { @@ -54,4 +55,26 @@ protected static InputStream getScriptAsStream(URI scriptPath) { } return is; } + + /** + * Start TajoScriptEngine. + * + * @param queryContext + * @throws IOException + */ + public abstract void start(OverridableConf queryContext) throws IOException; + + /** + * Shutdown TajoScriptEngine. + * @throws IOException + */ + public abstract void shutdown() throws IOException; + + /** + * Evaluate the input tuple. + * + * @param input + * @return + */ + public abstract Datum eval(Tuple input); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index d474ffb24b..3ca76ee01a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -35,7 +35,7 @@ import org.apache.tajo.exception.InternalException; import org.apache.tajo.plan.expr.*; import org.apache.tajo.plan.function.AggFunction; -import org.apache.tajo.plan.function.python.PythonScriptExecutor; +import org.apache.tajo.plan.function.python.PythonScriptEngine; import org.apache.tajo.plan.logical.WindowSpec; import org.apache.tajo.plan.serder.PlanProto.WinFunctionEvalSpec; @@ -183,7 +183,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { if (type == EvalType.FUNCTION) { current = new GeneralFunctionEval(context, funcDesc, params); if (evalContext != null && funcDesc.getInvocation().hasPython()) { - evalContext.addScriptExecutor(current, new PythonScriptExecutor(funcDesc)); + evalContext.addScriptEngine(current, new PythonScriptEngine(funcDesc)); } } else if (type == EvalType.AGG_FUNCTION || type == EvalType.WINDOW_FUNCTION) { AggFunction instance = (AggFunction) funcDesc.newInstance(); From 013ef2d95ede1b03e41c13a5c12d85a347720c6a Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Sun, 12 Apr 2015 13:09:27 +0900 Subject: [PATCH 48/55] TAJO-1344_3 --- .../tajo/plan/function/python/PythonScriptEngine.java | 10 ++++++++-- .../tajo/plan/function/python/TajoScriptEngine.java | 5 ++++- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index 4faf7a1b68..bc3558f726 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -44,6 +44,12 @@ import java.util.Set; import java.util.regex.Pattern; +/** + * {@link PythonScriptEngine} is responsible for registering python functions and maintaining the controller process. + * The controller is a python process that executes the python UDFs. + * (Please refer to 'tajo-core/src/main/resources/python/controller.py') + * Data are exchanged via standard I/O between PythonScriptEngine and the controller. + */ public class PythonScriptEngine extends TajoScriptEngine { private static final Log LOG = LogFactory.getLog(PythonScriptEngine.class); @@ -211,7 +217,7 @@ public void start(OverridableConf queryContext) throws IOException { startUdfController(); createInputHandlers(); setStreams(); - LOG.info("PythonScriptExecutor is started"); + LOG.info("PythonScriptExecutor starts up"); } public void shutdown() throws IOException { @@ -227,7 +233,7 @@ public void shutdown() throws IOException { } inputHandler.close(process); outputHandler.close(); - LOG.info("PythonScriptExecutor is shutdowned"); + LOG.info("PythonScriptExecutor shuts down"); } private void startUdfController() throws IOException { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 1d9e9c926e..ade173f8f7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -25,6 +25,9 @@ import java.io.*; import java.net.URI; +/** + * Abstract class of script engine + */ public abstract class TajoScriptEngine { /** @@ -71,7 +74,7 @@ protected static InputStream getScriptAsStream(URI scriptPath) { public abstract void shutdown() throws IOException; /** - * Evaluate the input tuple. + * Evaluate the input tuple. * * @param input * @return From 447f79f0e57079820d9455669992ff9a93d15d6b Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 16 Apr 2015 12:56:34 +0900 Subject: [PATCH 49/55] Fix findbugs warning --- .../tajo/function/PythonInvocationDesc.java | 4 ++-- .../tajo/catalog/store/AbstractDBStore.java | 2 -- .../org/apache/tajo/plan/ExprAnnotator.java | 21 ++++++++++--------- .../tajo/plan/function/FunctionInvoke.java | 2 +- .../function/LegacyScalarFunctionInvoke.java | 2 +- .../plan/function/PythonFunctionInvoke.java | 8 ++++++- .../function/python/PythonScriptEngine.java | 10 ++------- 7 files changed, 24 insertions(+), 25 deletions(-) diff --git a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java index f38735ac20..160b1692dd 100644 --- a/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java +++ b/tajo-catalog/tajo-catalog-common/src/main/java/org/apache/tajo/function/PythonInvocationDesc.java @@ -91,8 +91,8 @@ public String toString() { @Override public Object clone() throws CloneNotSupportedException { PythonInvocationDesc clone = (PythonInvocationDesc) super.clone(); - clone.funcName = funcName == null ? null : new String(funcName); - clone.filePath = filePath == null ? null : new String(filePath); + clone.funcName = funcName == null ? null : funcName; + clone.filePath = filePath == null ? null : filePath; return clone; } } diff --git a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java index a7a2fb04f6..b10e5a50d4 100644 --- a/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java +++ b/tajo-catalog/tajo-catalog-server/src/main/java/org/apache/tajo/catalog/store/AbstractDBStore.java @@ -61,8 +61,6 @@ public abstract class AbstractDBStore extends CatalogConstants implements Catalo protected final String catalogUri; private Connection conn; - - protected Map baseTableMaps = new HashMap(); protected XMLCatalogSchemaManager catalogSchemaManager; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index 0de561076f..0c5a01263d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -19,6 +19,7 @@ package org.apache.tajo.plan; import com.google.common.collect.Sets; +import org.apache.commons.collections.set.UnmodifiableSet; import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; import org.apache.tajo.algebra.*; @@ -72,7 +73,6 @@ static class Context { LogicalPlan plan; LogicalPlan.QueryBlock currentBlock; NameResolvingMode columnRsvLevel; - EvalContext evalContext; public Context(LogicalPlanner.PlanContext planContext, NameResolvingMode colRsvLevel) { this.queryContext = planContext.queryContext; @@ -385,7 +385,7 @@ public EvalNode visitValueListExpr(Context ctx, Stack stack, ValueListExpr if (!EvalTreeUtil.checkIfCanBeConstant(evalNodes[i])) { throw new PlanningException("Non constant values cannot be included in IN PREDICATE."); } - values[i] = EvalTreeUtil.evaluateImmediately(ctx.evalContext, evalNodes[i]); + values[i] = EvalTreeUtil.evaluateImmediately(null, evalNodes[i]); } return new RowConstantEval(values); } @@ -682,7 +682,8 @@ public EvalNode visitGeneralSetFunction(Context ctx, Stack stack, GeneralS } public static final Set WINDOW_FUNCTIONS = - Sets.newHashSet("row_number", "rank", "dense_rank", "percent_rank", "cume_dist", "first_value", "lag"); + UnmodifiableSet.decorate( + Sets.newHashSet("row_number", "rank", "dense_rank", "percent_rank", "cume_dist", "first_value", "lag")); public EvalNode visitWindowFunction(Context ctx, Stack stack, WindowFunctionExpr windowFunc) throws PlanningException { @@ -907,9 +908,9 @@ public EvalNode visitTimeLiteral(Context ctx, Stack stack, TimeLiteral exp public static int [] dateToIntArray(String years, String months, String days) throws PlanningException { - int year = Integer.valueOf(years); - int month = Integer.valueOf(months); - int day = Integer.valueOf(days); + int year = Integer.parseInt(years); + int month = Integer.parseInt(months); + int day = Integer.parseInt(days); if (!(1 <= year && year <= 9999)) { throw new PlanningException(String.format("Years (%d) must be between 1 and 9999 integer value", year)); @@ -933,12 +934,12 @@ public EvalNode visitTimeLiteral(Context ctx, Stack stack, TimeLiteral exp public static int [] timeToIntArray(String hours, String minutes, String seconds, String fractionOfSecond) throws PlanningException { - int hour = Integer.valueOf(hours); - int minute = Integer.valueOf(minutes); - int second = Integer.valueOf(seconds); + int hour = Integer.parseInt(hours); + int minute = Integer.parseInt(minutes); + int second = Integer.parseInt(seconds); int fraction = 0; if (fractionOfSecond != null) { - fraction = Integer.valueOf(fractionOfSecond); + fraction = Integer.parseInt(fractionOfSecond); } if (!(0 <= hour && hour <= 23)) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java index ea5e3c6335..728ae10fcc 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/FunctionInvoke.java @@ -33,7 +33,7 @@ * An abstract class for actual function invocation. * The metadata for function invocation are stored in the {@link org.apache.tajo.function.FunctionInvocation} class. */ -public abstract class FunctionInvoke { +public abstract class FunctionInvoke implements Cloneable { @Expose protected FunctionDesc functionDesc; public FunctionInvoke() { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java index 08e1368ece..6b2c116406 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/LegacyScalarFunctionInvoke.java @@ -29,7 +29,7 @@ /** * This class invokes the legacy scala functions. */ -public class LegacyScalarFunctionInvoke extends FunctionInvoke { +public class LegacyScalarFunctionInvoke extends FunctionInvoke implements Cloneable { @Expose private GeneralFunction function; public LegacyScalarFunctionInvoke() { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java index e6ab229c83..1019c60535 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/PythonFunctionInvoke.java @@ -30,7 +30,7 @@ */ public class PythonFunctionInvoke extends FunctionInvoke implements Cloneable { - private PythonScriptEngine scriptEngine; + private transient PythonScriptEngine scriptEngine; public PythonFunctionInvoke() { @@ -50,4 +50,10 @@ public Datum eval(Tuple tuple) { Datum res = scriptEngine.eval(tuple); return res; } + + @Override + public Object clone() throws CloneNotSupportedException { + // nothing to do + return super.clone(); + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index bc3558f726..70e3da022b 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -104,13 +104,11 @@ private static class FuncInfo { String returnType; String funcName; int paramNum; - int schemaLineNumber; - public FuncInfo(String returnType, String funcName, int paramNum, int schemaLineNumber) { + public FuncInfo(String returnType, String funcName, int paramNum) { this.returnType = returnType.toUpperCase(); this.funcName = funcName; this.paramNum = paramNum; - this.schemaLineNumber = schemaLineNumber; } } @@ -121,14 +119,11 @@ private static List getFunctions(InputStream is) throws IOException { BufferedReader br = new BufferedReader(in); String line = br.readLine(); String schemaString = null; - int lineNumber = 1; - int schemaLineNumber = -1; while (line != null) { if (pSchema.matcher(line).matches()) { int start = line.indexOf("(") + 2; //drop brackets/quotes int end = line.lastIndexOf(")") - 1; schemaString = line.substring(start,end).trim(); - schemaLineNumber = lineNumber; } else if (pDef.matcher(line).matches()) { int nameStart = line.indexOf("def ") + "def ".length(); int nameEnd = line.indexOf('('); @@ -143,11 +138,10 @@ private static List getFunctions(InputStream is) throws IOException { String functionName = line.substring(nameStart, nameEnd).trim(); schemaString = schemaString == null ? "blob" : schemaString; - functions.add(new FuncInfo(schemaString, functionName, paramNum, schemaLineNumber)); + functions.add(new FuncInfo(schemaString, functionName, paramNum)); schemaString = null; } line = br.readLine(); - lineNumber++; } br.close(); in.close(); From caa60b5161f3564a10d899b6cfa94907695381b1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 16 Apr 2015 13:06:17 +0900 Subject: [PATCH 50/55] Fix findbug warnings --- tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java index 64ccc49884..b8b768bd0c 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java @@ -31,7 +31,7 @@ import static org.apache.tajo.common.TajoDataTypes.DataType; -public class CastEval extends UnaryEval { +public class CastEval extends UnaryEval implements Cloneable { @Expose private DataType target; @Expose private TimeZone timezone; From 17969bf6ad37a634d1a56e1fcfc3e0e269e86dab Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 16 Apr 2015 13:53:10 +0900 Subject: [PATCH 51/55] TAJO-1344_3 --- .../java/org/apache/tajo/util/FileUtil.java | 23 +++++++++++ .../tajo/master/exec/QueryExecutor.java | 2 +- .../java/org/apache/tajo/worker/Task.java | 20 ++-------- .../function/python/PythonScriptEngine.java | 29 +++++++------- .../function/python/TajoScriptEngine.java | 2 +- .../plan/function/stream/InputHandler.java | 38 ++++--------------- .../plan/function/stream/OutputHandler.java | 5 ++- 7 files changed, 56 insertions(+), 63 deletions(-) diff --git a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java index 9403a2f4dd..3e3d3a2f59 100644 --- a/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java +++ b/tajo-common/src/main/java/org/apache/tajo/util/FileUtil.java @@ -19,6 +19,7 @@ package org.apache.tajo.util; import com.google.protobuf.Message; +import org.apache.commons.logging.Log; import org.apache.hadoop.fs.*; import org.apache.hadoop.io.IOUtils; @@ -143,4 +144,26 @@ public static String humanReadableByteCount(long bytes, boolean si) { public static boolean isLocalPath(Path path) { return path.toUri().getScheme().equals("file"); } + + + /** + * Close the Closeable objects and ignore any {@link IOException} or + * null pointers. Must only be used for cleanup in exception handlers. + * + * @param log the log to record problems to at debug level. Can be null. + * @param closeables the objects to close + */ + public static void cleanup(Log log, java.io.Closeable... closeables) { + for (java.io.Closeable c : closeables) { + if (c != null) { + try { + c.close(); + } catch(IOException e) { + if (log != null && log.isDebugEnabled()) { + log.debug("Exception in closing " + c, e); + } + } + } + } + } } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index d81660b070..20a812f367 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -322,7 +322,7 @@ public static void startScriptExecutors(QueryContext queryContext, EvalContext e } } - public static void stopScriptExecutors(EvalContext evalContext) throws IOException { + public static void stopScriptExecutors(EvalContext evalContext) { for (TajoScriptEngine executor : evalContext.getAllScriptEngines()) { executor.shutdown(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index d432520108..0a8cdc0cfb 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -200,7 +200,7 @@ private void startScriptExecutors() throws IOException { } } - private void stopScriptExecutors() throws IOException { + private void stopScriptExecutors() { for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { executor.shutdown(); } @@ -274,21 +274,13 @@ public void fetch() { } public void kill() { - try { - stopScriptExecutors(); - } catch (IOException e) { - throw new RuntimeException(e); - } + stopScriptExecutors(); context.setState(TaskAttemptState.TA_KILLED); context.stop(); } public void abort() { - try { - stopScriptExecutors(); - } catch (IOException e) { - throw new RuntimeException(e); - } + stopScriptExecutors(); context.stop(); } @@ -662,11 +654,7 @@ public void run() { if (retryNum == maxRetryNum) { LOG.error("ERROR: the maximum retry (" + retryNum + ") on the fetch exceeded (" + fetcher.getURI() + ")"); } - try { - stopScriptExecutors(); - } catch (IOException e) { - throw new RuntimeException(e); - } + stopScriptExecutors(); context.stop(); // retry task ctx.getFetchLatch().countDown(); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index 70e3da022b..d081665fa8 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -35,6 +35,7 @@ import org.apache.tajo.plan.function.stream.*; import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; +import org.apache.tajo.util.FileUtil; import org.apache.tajo.util.TUtil; import java.io.*; @@ -211,23 +212,25 @@ public void start(OverridableConf queryContext) throws IOException { startUdfController(); createInputHandlers(); setStreams(); - LOG.info("PythonScriptExecutor starts up"); + if (LOG.isDebugEnabled()) { + LOG.debug("PythonScriptExecutor starts up"); + } } - public void shutdown() throws IOException { + public void shutdown() { process.destroy(); - if (stdin != null) { - stdin.close(); - } - if (stdout != null) { - stdout.close(); - } - if (stderr != null) { - stderr.close(); + FileUtil.cleanup(LOG, stdin); + FileUtil.cleanup(LOG, stdout); + FileUtil.cleanup(LOG, stderr); + FileUtil.cleanup(LOG, inputHandler); + FileUtil.cleanup(LOG, outputHandler); + stdin = null; + stdout = stderr = null; + inputHandler = null; + outputHandler = null; + if (LOG.isDebugEnabled()) { + LOG.debug("PythonScriptExecutor shuts down"); } - inputHandler.close(process); - outputHandler.close(); - LOG.info("PythonScriptExecutor shuts down"); } private void startUdfController() throws IOException { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index ade173f8f7..2ba9c2b4d2 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -71,7 +71,7 @@ protected static InputStream getScriptAsStream(URI scriptPath) { * Shutdown TajoScriptEngine. * @throws IOException */ - public abstract void shutdown() throws IOException; + public abstract void shutdown(); /** * Evaluate the input tuple. diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java index 12e341a61a..dcc53c1749 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/InputHandler.java @@ -20,6 +20,7 @@ import org.apache.tajo.storage.Tuple; +import java.io.Closeable; import java.io.IOException; import java.io.OutputStream; @@ -27,7 +28,7 @@ * {@link InputHandler} is responsible for handling the input to the Tajo-Streaming external command. * */ -public class InputHandler { +public class InputHandler implements Closeable { private final static byte[] END_OF_RECORD_DELIM = "|_\n".getBytes(); private final static byte[] END_OF_STREAM = ("C" + "\\x04" + "|_\n").getBytes(); @@ -54,35 +55,12 @@ public void putNext(Tuple t) throws IOException { out.write(END_OF_RECORD_DELIM); } - /** - * Close the InputHandler since there is no more input - * to be sent to the managed process. - * @param process the managed process - this could be null in some cases - * like when input is through files. In that case, the process would not - * have been exec'ed yet - if this method if overridden it is the responsibility - * of the implementer to check that the process is usable. The managed process - * object is supplied by the ExecutableManager to this call so that this method - * can check if the process is alive if it needs to know. - * - * @throws IOException - */ - public synchronized void close(Process process) throws IOException { - try { - if (!alreadyClosed) { - alreadyClosed = true; - out.flush(); - out.close(); - out = null; - } - } catch(IOException e) { - // check if we got an exception because - // the process actually completed and we were - // trying to flush and close it's stdin - if (process == null || process.exitValue() != 0) { - // the process had not terminated normally - // throw the exception we got - throw e; - } + public void close() throws IOException { + if (!alreadyClosed) { + out.flush(); + out.close(); + out = null; + alreadyClosed = true; } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java index 9e64288808..c5eb4197db 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/OutputHandler.java @@ -23,6 +23,7 @@ import org.apache.tajo.storage.Tuple; import org.apache.tajo.storage.VTuple; +import java.io.Closeable; import java.io.IOException; import java.io.InputStream; @@ -30,7 +31,7 @@ * {@link OutputHandler} is responsible for handling the output of the * Tajo-Streaming external command. */ -public class OutputHandler { +public class OutputHandler implements Closeable { private static int DEFAULT_BUFFER = 64 * 1024; private final static byte[] END_OF_RECORD_DELIM = "|_\n".getBytes(); @@ -145,7 +146,7 @@ private boolean isEndOfRow() { * Close the OutputHandler. * @throws IOException */ - public synchronized void close() throws IOException { + public void close() throws IOException { if(!alreadyClosed) { istream.close(); istream = null; From a74c213a504b99eaaf1cd7616424402b3acb7f26 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Thu, 16 Apr 2015 13:54:58 +0900 Subject: [PATCH 52/55] TAJO-1344_3 --- .../apache/tajo/plan/function/stream/ByteBufInputChannel.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java index 869ebe9c2f..daf2357dca 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/stream/ByteBufInputChannel.java @@ -19,7 +19,7 @@ package org.apache.tajo.plan.function.stream; import org.apache.hadoop.fs.ByteBufferReadable; -import org.apache.hadoop.io.IOUtils; +import org.apache.tajo.util.FileUtil; import java.io.IOException; import java.io.InputStream; @@ -66,6 +66,6 @@ public int read(ByteBuffer dst) throws IOException { @Override protected void implCloseChannel() throws IOException { - IOUtils.cleanup(null, channel, inputStream); + FileUtil.cleanup(null, channel, inputStream); } } From 43e687d36ab6a9961a274d78a91ff94f234cefc1 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 17 Apr 2015 16:50:31 +0900 Subject: [PATCH 53/55] TAJO-1344_3 --- .../main/java/org/apache/tajo/QueryVars.java | 2 - .../java/org/apache/tajo/conf/TajoConf.java | 1 + .../tajo/engine/function/FunctionLoader.java | 6 +- .../rules/GlobalPlanEqualityTester.java | 2 +- .../tajo/engine/query/QueryContext.java | 8 -- .../org/apache/tajo/master/GlobalEngine.java | 2 - .../org/apache/tajo/master/TajoMaster.java | 2 +- .../tajo/master/exec/QueryExecutor.java | 4 +- .../apache/tajo/util/QueryContextUtil.java | 28 ----- .../org/apache/tajo/worker/TajoWorker.java | 2 +- .../java/org/apache/tajo/worker/Task.java | 9 +- .../tajo/worker/TaskAttemptContext.java | 3 +- .../apache/tajo/engine/eval/ExprTestBase.java | 12 +- .../org/apache/tajo/plan/ExprAnnotator.java | 6 +- .../tajo/plan/expr/BetweenPredicateEval.java | 3 +- .../org/apache/tajo/plan/expr/CastEval.java | 19 +-- .../apache/tajo/plan/expr/EvalContext.java | 11 ++ .../org/apache/tajo/plan/expr/FieldEval.java | 3 +- .../apache/tajo/plan/expr/FunctionEval.java | 3 +- .../tajo/plan/expr/GeneralFunctionEval.java | 8 +- .../plan/expr/PatternMatchPredicateEval.java | 3 +- .../exprrewrite/rules/ConstantFolding.java | 4 +- .../function/python/PythonScriptEngine.java | 19 +-- .../function/python/TajoScriptEngine.java | 6 +- .../rules/LogicalPlanEqualityTester.java | 2 +- .../plan/serder/EvalNodeDeserializer.java | 6 +- .../plan/serder/LogicalNodeDeserializer.java | 108 +++++++++--------- 27 files changed, 129 insertions(+), 153 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java index c3835f8bfb..55ca7009f0 100644 --- a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java +++ b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java @@ -29,8 +29,6 @@ public enum QueryVars implements ConfigKey { OUTPUT_OVERWRITE, OUTPUT_AS_DIRECTORY, OUTPUT_PER_FILE_SIZE, - PYTHON_SCRIPT_CODE_DIR, - PYTHON_CONTROLLER_LOG_DIR ; QueryVars() { diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 84ee05999c..bfba290e9c 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -295,6 +295,7 @@ public static enum ConfVars implements ConfigKey { // Python UDF PYTHON_CODE_DIR("tajo.function.python.code-dir", ""), + PYTHON_CONTROLLER_LOG_DIR("tajo.function.python.controller.log-dir", ""), ///////////////////////////////////////////////////////////////////////////////// // User Session Configuration diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index c259e7b89f..d9abc609be 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -84,15 +84,15 @@ public static Map load() { } /** - * Load functions that are optionally defined by users. + * Load functions that are user-defined functions. * * @param conf * @param functionMap * @return * @throws IOException */ - public static Map loadOptionalFunctions(TajoConf conf, - Map functionMap) + public static Map loadUserDefinedFunctions(TajoConf conf, + Map functionMap) throws IOException { String[] codePaths = conf.getStrings(TajoConf.ConfVars.PYTHON_CODE_DIR.varname); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java index e55a258b4e..d0c548f4c5 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java @@ -51,7 +51,7 @@ public MasterPlan rewrite(MasterPlan plan) { LogicalNode node = eb.getPlan(); if (node != null) { PlanProto.LogicalNodeTree tree = LogicalNodeSerializer.serialize(node); - LogicalNode deserialize = LogicalNodeDeserializer.deserialize(plan.getContext(), null, tree); + LogicalNode deserialize = LogicalNodeDeserializer.deserialize(null, tree); assert node.deepEquals(deserialize); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java index 0e595f79f3..7b3c00db30 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java @@ -151,14 +151,6 @@ public NodeType getCommandType() { return strVal != null ? NodeType.valueOf(strVal) : null; } - public void setPythonScriptPath(String path) { - put(QueryVars.PYTHON_SCRIPT_CODE_DIR, path); - } - - public String getPythonScriptPath() { - return get(QueryVars.PYTHON_SCRIPT_CODE_DIR); - } - public void setCreateTable() { setCommandType(NodeType.CREATE_TABLE); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java index 04a0bad076..074f34e94c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java @@ -56,7 +56,6 @@ import org.apache.tajo.plan.verifier.VerifyException; import org.apache.tajo.storage.StorageManager; import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.QueryContextUtil; import java.io.IOException; import java.sql.SQLException; @@ -174,7 +173,6 @@ public Expr load(String sql) throws SQLSyntaxError { public SubmitQueryResponse executeQuery(Session session, String query, boolean isJson) { LOG.info("Query: " + query); QueryContext queryContext = createQueryContext(session); - QueryContextUtil.updatePythonScriptPath(context.getConf(), queryContext); Expr planningContext; try { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 5d88ea0554..0a5de58897 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -210,7 +210,7 @@ public void serviceInit(Configuration _conf) throws Exception { private Collection loadFunctions() throws IOException { Map functionMap = FunctionLoader.load(); - return FunctionLoader.loadOptionalFunctions(systemConf, functionMap).values(); + return FunctionLoader.loadUserDefinedFunctions(systemConf, functionMap).values(); } private void initSystemMetrics() { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index 20a812f367..90bb774a64 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -269,7 +269,7 @@ public void execNonFromQuery(QueryContext queryContext, LogicalPlan plan, Submit throws Exception { LogicalRootNode rootNode = plan.getRootBlock().getRoot(); - EvalContext evalContext = new EvalContext(); + EvalContext evalContext = new EvalContext(queryContext); Target[] targets = plan.getRootBlock().getRawTargets(); if (targets == null) { throw new PlanningException("No targets"); @@ -316,7 +316,7 @@ public static void startScriptExecutors(QueryContext queryContext, EvalContext e if (functionEval.getFuncDesc().getInvocation().hasPython()) { TajoScriptEngine scriptExecutor = new PythonScriptEngine(functionEval.getFuncDesc()); evalContext.addScriptEngine(eval, scriptExecutor); - scriptExecutor.start(queryContext); + scriptExecutor.start(queryContext.getConf()); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java deleted file mode 100644 index dbee78e5ba..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.util; - -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.query.QueryContext; - -public class QueryContextUtil { - public static void updatePythonScriptPath(TajoConf systemConf, QueryContext queryContext) { - queryContext.setPythonScriptPath(systemConf.getVar(TajoConf.ConfVars.PYTHON_CODE_DIR)); - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index d66474516d..17af71a5a4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -237,7 +237,7 @@ public void serviceInit(Configuration conf) throws Exception { historyReader = new HistoryReader(workerContext.getWorkerName(), this.systemConf); - FunctionLoader.loadOptionalFunctions(systemConf, new HashMap()); + FunctionLoader.loadUserDefinedFunctions(systemConf, new HashMap()); diagnoseTajoWorker(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index db0833fa89..76283d7022 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -46,7 +46,6 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.*; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.function.python.PythonScriptEngine; import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.serder.LogicalNodeDeserializer; @@ -58,7 +57,6 @@ import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.NetUtils; -import org.apache.tajo.util.QueryContextUtil; import java.io.File; import java.io.IOException; @@ -102,8 +100,6 @@ public class Task { private Schema finalSchema = null; private TupleComparator sortComp = null; - private PythonScriptEngine pythonEngine; - public Task(String taskRunnerId, Path baseDir, TaskAttemptId taskId, @@ -138,8 +134,7 @@ public Task(String taskRunnerId, } public void initPlan() throws IOException { - QueryContextUtil.updatePythonScriptPath(systemConf, queryContext); - plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); + plan = LogicalNodeDeserializer.deserialize(context.getEvalContext(), request.getPlan()); LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); if (scanNode != null) { for (LogicalNode node : scanNode) { @@ -196,7 +191,7 @@ public void initPlan() throws IOException { private void startScriptExecutors() throws IOException { for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.start(queryContext); + executor.start(systemConf); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java index 58028ac014..72eacd0c74 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java @@ -84,13 +84,14 @@ public class TaskAttemptContext { private Map partitionOutputVolume; private HashShuffleAppenderManager hashShuffleAppenderManager; - private EvalContext evalContext = new EvalContext(); + private EvalContext evalContext; public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext executionBlockContext, final TaskAttemptId queryId, final FragmentProto[] fragments, final Path workDir) { this.queryContext = queryContext; + this.evalContext = new EvalContext(queryContext); if (executionBlockContext != null) { // For unit tests this.workerContext = executionBlockContext.getWorkerContext(); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 9d6b1121a0..8886bf23eb 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -55,7 +55,6 @@ import org.apache.tajo.util.BytesUtils; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.KeyValueSet; -import org.apache.tajo.util.QueryContextUtil; import org.apache.tajo.util.datetime.DateTimeUtil; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -97,7 +96,7 @@ public static void setUp() throws Exception { cat.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); cat.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); Map map = FunctionLoader.load(); - map = FunctionLoader.loadOptionalFunctions(conf, map); + map = FunctionLoader.loadUserDefinedFunctions(conf, map); for (FunctionDesc funcDesc : map.values()) { cat.createFunction(funcDesc); } @@ -174,7 +173,7 @@ private static Target[] getRawTargets(QueryContext context, String query, boolea assertJsonSerDer(t.getEvalTree()); } for (Target t : targets) { - assertEvalTreeProtoSerDer(context, t.getEvalTree()); + assertEvalTreeProtoSerDer(t.getEvalTree()); } return targets; } @@ -226,7 +225,6 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S queryContext = LocalTajoTestingUtility.createDummyContext(conf); queryContext.putAll(context); } - QueryContextUtil.updatePythonScriptPath(conf, queryContext); String timezoneId = queryContext.get(SessionVars.TIMEZONE); TimeZone timeZone = TimeZone.getTimeZone(timezoneId); @@ -273,7 +271,7 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S Target [] targets; TajoClassLoader classLoader = new TajoClassLoader(); - EvalContext evalContext = new EvalContext(); + EvalContext evalContext = new EvalContext(queryContext); try { targets = getRawTargets(queryContext, query, condition); @@ -332,8 +330,8 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S } } - public static void assertEvalTreeProtoSerDer(OverridableConf context, EvalNode evalNode) { + public static void assertEvalTreeProtoSerDer(EvalNode evalNode) { PlanProto.EvalNodeTree converted = EvalNodeSerializer.serialize(evalNode); - assertEquals(evalNode, EvalNodeDeserializer.deserialize(context, null, converted)); + assertEquals(evalNode, EvalNodeDeserializer.deserialize(null, converted)); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index 0c5a01263d..564975816d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -188,7 +188,7 @@ private static EvalNode convertType(Context ctx, EvalNode evalNode, DataType toT return newConst; } else { - return new CastEval(ctx.queryContext, evalNode, toType); + return new CastEval(evalNode, toType); } } @@ -609,7 +609,7 @@ public EvalNode visitFunction(Context ctx, Stack stack, FunctionExpr expr) FunctionType functionType = funcDesc.getFuncType(); if (functionType == FunctionType.GENERAL || functionType == FunctionType.UDF) { - return new GeneralFunctionEval(ctx.queryContext, funcDesc, givenArgs); + return new GeneralFunctionEval(funcDesc, givenArgs); } else if (functionType == FunctionType.AGGREGATION || functionType == FunctionType.UDA) { if (!ctx.currentBlock.hasNode(NodeType.GROUP_BY)) { @@ -801,7 +801,7 @@ public EvalNode visitCastExpr(Context ctx, Stack stack, CastExpr expr) thr LogicalPlanner.convertDataType(expr.getTarget()).getDataType(), tz)); } else { - return new CastEval(ctx.queryContext, child, LogicalPlanner.convertDataType(expr.getTarget()).getDataType()); + return new CastEval(child, LogicalPlanner.convertDataType(expr.getTarget()).getDataType()); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java index eb3e36bd2f..6002c185e7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java @@ -20,6 +20,7 @@ import com.google.gson.annotations.Expose; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes; @@ -227,7 +228,7 @@ public String toString() { } @Override - public EvalNode bind(EvalContext evalContext, Schema schema) { + public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); if (begin.getType() == EvalType.CONST && end.getType() == EvalType.CONST) { Datum beginValue = ((ConstEval)begin).getValue(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java index b8b768bd0c..b0cce3c094 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java @@ -20,8 +20,9 @@ import com.google.gson.annotations.Expose; -import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; +import org.apache.tajo.annotation.Nullable; +import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.storage.Tuple; @@ -35,14 +36,9 @@ public class CastEval extends UnaryEval implements Cloneable { @Expose private DataType target; @Expose private TimeZone timezone; - public CastEval(OverridableConf context, EvalNode operand, DataType target) { + public CastEval(EvalNode operand, DataType target) { super(EvalType.CAST, operand); this.target = target; - - if (context.containsKey(SessionVars.TIMEZONE)) { - String timezoneId = context.get(SessionVars.TIMEZONE); - timezone = TimeZone.getTimeZone(timezoneId); - } } public EvalNode getOperand() { @@ -67,6 +63,15 @@ public String getName() { return target.getType().name(); } + public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { + super.bind(evalContext, schema); + if (evalContext.getQueryContext().containsKey(SessionVars.TIMEZONE)) { + String timezoneId = evalContext.getQueryContext().get(SessionVars.TIMEZONE); + timezone = TimeZone.getTimeZone(timezoneId); + } + return this; + } + @Override @SuppressWarnings("unchecked") public Datum eval(Tuple tuple) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java index 6a30e772ce..928303bfd0 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java @@ -18,6 +18,7 @@ package org.apache.tajo.plan.expr; +import org.apache.tajo.OverridableConf; import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.util.TUtil; @@ -25,8 +26,14 @@ import java.util.Map; public class EvalContext { + private final OverridableConf queryContext; + private final Map scriptEngineMap = TUtil.newHashMap(); + public EvalContext(OverridableConf queryContext) { + this.queryContext = queryContext; + } + public void addScriptEngine(EvalNode evalNode, TajoScriptEngine scriptExecutor) { this.scriptEngineMap.put(evalNode, scriptExecutor); } @@ -42,4 +49,8 @@ public TajoScriptEngine getScriptEngine(EvalNode evalNode) { public Collection getAllScriptEngines() { return this.scriptEngineMap.values(); } + + public OverridableConf getQueryContext() { + return queryContext; + } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java index 870970b538..15f916e743 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java @@ -19,6 +19,7 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes.DataType; @@ -40,7 +41,7 @@ public FieldEval(Column column) { } @Override - public EvalNode bind(EvalContext evalContext, Schema schema) { + public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); // TODO - column namespace should be improved to simplify name handling and resolving. if (column.hasQualifier()) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java index dd9121bd5a..1dc26e56e1 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java @@ -21,6 +21,7 @@ import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.gson.annotations.Expose; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes; @@ -49,7 +50,7 @@ public FunctionEval(EvalType type, FunctionDesc funcDesc, EvalNode[] argEvals) { } @Override - public EvalNode bind(EvalContext evalContext, Schema schema) { + public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); this.params = new VTuple(argEvals.length); return this; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 30fbe910f5..81a4fcc54a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -20,6 +20,7 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.OverridableConf; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; @@ -33,15 +34,16 @@ public class GeneralFunctionEval extends FunctionEval { protected FunctionInvoke funcInvoke; @Expose protected FunctionInvokeContext invokeContext; - public GeneralFunctionEval(OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) + public GeneralFunctionEval(FunctionDesc desc, EvalNode[] givenArgs) throws IOException { super(EvalType.FUNCTION, desc, givenArgs); - this.invokeContext = new FunctionInvokeContext(queryContext, getParamType()); } @Override - public EvalNode bind(EvalContext evalContext, Schema schema) { + public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); + + this.invokeContext = new FunctionInvokeContext(evalContext.getQueryContext(), getParamType()); try { this.funcInvoke = FunctionInvoke.newInstance(funcDesc); if (evalContext != null && evalContext.hasScriptEngine(this)) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java index cdd8dfb575..871c2ab66c 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java @@ -19,6 +19,7 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; +import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes; @@ -74,7 +75,7 @@ public String getName() { } @Override - public EvalNode bind(EvalContext evalContext, Schema schema) { + public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); compile(pattern); return this; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index 942c3b47cd..5e88ae4f0a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -94,8 +94,8 @@ public EvalNode visitFuncCall(LogicalPlanner.PlanContext context, FunctionEval e if (evalNode.getFuncDesc().getInvocation().hasPython()) { TajoScriptEngine executor = new PythonScriptEngine(evalNode.getFuncDesc()); try { - executor.start(context.getQueryContext()); - EvalContext evalContext = new EvalContext(); + executor.start(context.getQueryContext().getConf()); + EvalContext evalContext = new EvalContext(context.getQueryContext()); evalContext.addScriptEngine(evalNode, executor); evalNode.bind(evalContext, null); Datum funcRes = evalNode.eval(null); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index d081665fa8..15eac6b2b6 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -21,6 +21,7 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; import org.apache.tajo.OverridableConf; import org.apache.tajo.QueryVars; import org.apache.tajo.catalog.*; @@ -170,7 +171,7 @@ private static List getFunctions(InputStream is) throws IOException { private static final int CONTROLLER_LOG_FILE_PATH = 8; // Controller log file logs progress through the controller script not user code. private static final int OUT_SCHEMA = 9; // the schema of the output column - private OverridableConf queryContext; + private Configuration systemConf; private Process process; // Handle to the external execution of python functions @@ -207,8 +208,9 @@ public PythonScriptEngine(FunctionDesc functionDesc) { outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); } - public void start(OverridableConf queryContext) throws IOException { - this.queryContext = queryContext; + @Override + public void start(Configuration systemConf) throws IOException { + this.systemConf = systemConf; startUdfController(); createInputHandlers(); setStreams(); @@ -217,6 +219,7 @@ public void start(OverridableConf queryContext) throws IOException { } } + @Override public void shutdown() { process.destroy(); FileUtil.cleanup(LOG, stdin); @@ -247,10 +250,10 @@ private String[] buildCommand() throws IOException { String[] command = new String[10]; // TODO: support controller logging - String standardOutputRootWriteLocation = ""; - if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { + String standardOutputRootWriteLocation = systemConf.get(TajoConf.ConfVars.PYTHON_CONTROLLER_LOG_DIR.keyname(), + DEFAULT_LOG_DIR); + if (standardOutputRootWriteLocation != null) { LOG.warn("Currently, logging is not supported for the python controller."); - standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR, DEFAULT_LOG_DIR); } String controllerLogFileName, outFileName, errOutFileName; @@ -269,10 +272,10 @@ private String[] buildCommand() throws IOException { command[UDF_FILE_NAME] = fileName; command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; - if (!queryContext.containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { + String fileCachePath = systemConf.get(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname()); + if (fileCachePath == null) { throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); } - String fileCachePath = queryContext.get(QueryVars.PYTHON_SCRIPT_CODE_DIR); command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; command[STD_OUT_OUTPUT_PATH] = outFileName; command[STD_ERR_OUTPUT_PATH] = errOutFileName; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 2ba9c2b4d2..726ec2faf7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -18,7 +18,7 @@ package org.apache.tajo.plan.function.python; -import org.apache.tajo.OverridableConf; +import org.apache.hadoop.conf.Configuration; import org.apache.tajo.datum.Datum; import org.apache.tajo.storage.Tuple; @@ -62,10 +62,10 @@ protected static InputStream getScriptAsStream(URI scriptPath) { /** * Start TajoScriptEngine. * - * @param queryContext + * @param systemConf * @throws IOException */ - public abstract void start(OverridableConf queryContext) throws IOException; + public abstract void start(Configuration systemConf) throws IOException; /** * Shutdown TajoScriptEngine. diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java index 43a86184b3..7ca4f59722 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java @@ -48,7 +48,7 @@ public boolean isEligible(OverridableConf queryContext, LogicalPlan plan) { public LogicalPlan rewrite(OverridableConf queryContext, LogicalPlan plan) throws PlanningException { LogicalNode root = plan.getRootBlock().getRoot(); PlanProto.LogicalNodeTree serialized = LogicalNodeSerializer.serialize(plan.getRootBlock().getRoot()); - LogicalNode deserialized = LogicalNodeDeserializer.deserialize(queryContext, null, serialized); + LogicalNode deserialized = LogicalNodeDeserializer.deserialize(null, serialized); assert root.deepEquals(deserialized); return plan; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index 3ca76ee01a..dae56d4332 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -53,7 +53,7 @@ */ public class EvalNodeDeserializer { - public static EvalNode deserialize(OverridableConf context, EvalContext evalContext, PlanProto.EvalNodeTree tree) { + public static EvalNode deserialize(EvalContext evalContext, PlanProto.EvalNodeTree tree) { Map evalNodeMap = Maps.newHashMap(); // sort serialized eval nodes in an ascending order of their IDs. @@ -89,7 +89,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { current = new IsNullEval(unaryProto.getNegative(), child); break; case CAST: - current = new CastEval(context, child, unaryProto.getCastingType()); + current = new CastEval(child, unaryProto.getCastingType()); break; case SIGNED: current = new SignedEval(unaryProto.getNegative(), child); @@ -181,7 +181,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { try { funcDesc = new FunctionDesc(funcProto.getFuncion()); if (type == EvalType.FUNCTION) { - current = new GeneralFunctionEval(context, funcDesc, params); + current = new GeneralFunctionEval(funcDesc, params); if (evalContext != null && funcDesc.getInvocation().hasPython()) { evalContext.addScriptEngine(current, new PythonScriptEngine(funcDesc)); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java index 84991bbf75..3101ca1189 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java @@ -21,7 +21,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.hadoop.fs.Path; -import org.apache.tajo.OverridableConf; import org.apache.tajo.algebra.JoinType; import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Column; @@ -52,12 +51,11 @@ public class LogicalNodeDeserializer { /** * Deserialize a list of nodes into a logical node tree. * - * @param context QueryContext + * @param evalContext eval context * @param tree LogicalNodeTree which contains a list of serialized logical nodes. * @return A logical node tree */ - public static LogicalNode deserialize(OverridableConf context, @Nullable EvalContext evalContext, - PlanProto.LogicalNodeTree tree) { + public static LogicalNode deserialize(@Nullable EvalContext evalContext, PlanProto.LogicalNodeTree tree) { Map nodeMap = Maps.newHashMap(); // sort serialized logical nodes in an ascending order of their sids @@ -87,10 +85,10 @@ public int compare(PlanProto.LogicalNode o1, PlanProto.LogicalNode o2) { current = convertSetSession(protoNode); break; case EXPRS: - current = convertEvalExpr(context, evalContext, protoNode); + current = convertEvalExpr(evalContext, protoNode); break; case PROJECTION: - current = convertProjection(context, evalContext, nodeMap, protoNode); + current = convertProjection(evalContext, nodeMap, protoNode); break; case LIMIT: current = convertLimit(nodeMap, protoNode); @@ -99,34 +97,34 @@ public int compare(PlanProto.LogicalNode o1, PlanProto.LogicalNode o2) { current = convertSort(nodeMap, protoNode); break; case WINDOW_AGG: - current = convertWindowAgg(context, evalContext, nodeMap, protoNode); + current = convertWindowAgg(evalContext, nodeMap, protoNode); break; case HAVING: - current = convertHaving(context, evalContext, nodeMap, protoNode); + current = convertHaving(evalContext, nodeMap, protoNode); break; case GROUP_BY: - current = convertGroupby(context, evalContext, nodeMap, protoNode); + current = convertGroupby(evalContext, nodeMap, protoNode); break; case DISTINCT_GROUP_BY: - current = convertDistinctGroupby(context, evalContext, nodeMap, protoNode); + current = convertDistinctGroupby(evalContext, nodeMap, protoNode); break; case SELECTION: - current = convertFilter(context, evalContext, nodeMap, protoNode); + current = convertFilter(evalContext, nodeMap, protoNode); break; case JOIN: - current = convertJoin(context, evalContext, nodeMap, protoNode); + current = convertJoin(evalContext, nodeMap, protoNode); break; case TABLE_SUBQUERY: - current = convertTableSubQuery(context, evalContext, nodeMap, protoNode); + current = convertTableSubQuery(evalContext, nodeMap, protoNode); break; case UNION: current = convertUnion(nodeMap, protoNode); break; case PARTITIONS_SCAN: - current = convertPartitionScan(context, evalContext, protoNode); + current = convertPartitionScan(evalContext, protoNode); break; case SCAN: - current = convertScan(context, evalContext, protoNode); + current = convertScan(evalContext, protoNode); break; case CREATE_TABLE: @@ -191,24 +189,24 @@ private static SetSessionNode convertSetSession(PlanProto.LogicalNode protoNode) return setSession; } - private static EvalExprNode convertEvalExpr(OverridableConf context, EvalContext evalContext, + private static EvalExprNode convertEvalExpr(EvalContext evalContext, PlanProto.LogicalNode protoNode) { PlanProto.EvalExprNode evalExprProto = protoNode.getExprEval(); EvalExprNode evalExpr = new EvalExprNode(protoNode.getNodeId()); evalExpr.setInSchema(convertSchema(protoNode.getInSchema())); - evalExpr.setTargets(convertTargets(context, evalContext, evalExprProto.getTargetsList())); + evalExpr.setTargets(convertTargets(evalContext, evalExprProto.getTargetsList())); return evalExpr; } - private static ProjectionNode convertProjection(OverridableConf context, EvalContext evalContext, + private static ProjectionNode convertProjection(EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.ProjectionNode projectionProto = protoNode.getProjection(); ProjectionNode projectionNode = new ProjectionNode(protoNode.getNodeId()); - projectionNode.init(projectionProto.getDistinct(), convertTargets(context, evalContext, + projectionNode.init(projectionProto.getDistinct(), convertTargets(evalContext, projectionProto.getTargetsList())); projectionNode.setChild(nodeMap.get(projectionProto.getChildSeq())); projectionNode.setInSchema(convertSchema(protoNode.getInSchema())); @@ -241,21 +239,20 @@ private static SortNode convertSort(Map nodeMap, PlanProto return sortNode; } - private static HavingNode convertHaving(OverridableConf context, EvalContext evalContext, + private static HavingNode convertHaving(EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.FilterNode havingProto = protoNode.getFilter(); HavingNode having = new HavingNode(protoNode.getNodeId()); having.setChild(nodeMap.get(havingProto.getChildSeq())); - having.setQual(EvalNodeDeserializer.deserialize(context, evalContext, havingProto.getQual())); + having.setQual(EvalNodeDeserializer.deserialize(evalContext, havingProto.getQual())); having.setInSchema(convertSchema(protoNode.getInSchema())); having.setOutSchema(convertSchema(protoNode.getOutSchema())); return having; } - private static WindowAggNode convertWindowAgg(OverridableConf context, EvalContext evalContext, - Map nodeMap, + private static WindowAggNode convertWindowAgg(EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.WindowAggNode windowAggProto = protoNode.getWindowAgg(); @@ -267,7 +264,7 @@ private static WindowAggNode convertWindowAgg(OverridableConf context, EvalConte } if (windowAggProto.getWindowFunctionsCount() > 0) { - windowAgg.setWindowFunctions(convertWindowFunccEvals(context, evalContext, + windowAgg.setWindowFunctions(convertWindowFunccEvals(evalContext, windowAggProto.getWindowFunctionsList())); } @@ -278,7 +275,7 @@ private static WindowAggNode convertWindowAgg(OverridableConf context, EvalConte } if (windowAggProto.getTargetsCount() > 0) { - windowAgg.setTargets(convertTargets(context, evalContext, windowAggProto.getTargetsList())); + windowAgg.setTargets(convertTargets(evalContext, windowAggProto.getTargetsList())); } windowAgg.setInSchema(convertSchema(protoNode.getInSchema())); @@ -287,8 +284,8 @@ private static WindowAggNode convertWindowAgg(OverridableConf context, EvalConte return windowAgg; } - private static GroupbyNode convertGroupby(OverridableConf context, EvalContext evalContext, - Map nodeMap, PlanProto.LogicalNode protoNode) { + private static GroupbyNode convertGroupby(EvalContext evalContext, Map nodeMap, + PlanProto.LogicalNode protoNode) { PlanProto.GroupbyNode groupbyProto = protoNode.getGroupby(); GroupbyNode groupby = new GroupbyNode(protoNode.getNodeId()); @@ -299,10 +296,10 @@ private static GroupbyNode convertGroupby(OverridableConf context, EvalContext e groupby.setGroupingColumns(convertColumns(groupbyProto.getGroupingKeysList())); } if (groupbyProto.getAggFunctionsCount() > 0) { - groupby.setAggFunctions(convertAggFuncCallEvals(context, evalContext, groupbyProto.getAggFunctionsList())); + groupby.setAggFunctions(convertAggFuncCallEvals(evalContext, groupbyProto.getAggFunctionsList())); } if (groupbyProto.getTargetsCount() > 0) { - groupby.setTargets(convertTargets(context, evalContext, groupbyProto.getTargetsList())); + groupby.setTargets(convertTargets(evalContext, groupbyProto.getTargetsList())); } groupby.setInSchema(convertSchema(protoNode.getInSchema())); @@ -311,8 +308,7 @@ private static GroupbyNode convertGroupby(OverridableConf context, EvalContext e return groupby; } - private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf context, EvalContext evalContext, - Map nodeMap, + private static DistinctGroupbyNode convertDistinctGroupby(EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.DistinctGroupbyNode distinctGroupbyProto = protoNode.getDistinctGroupby(); @@ -320,14 +316,14 @@ private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf contex distinctGroupby.setChild(nodeMap.get(distinctGroupbyProto.getChildSeq())); if (distinctGroupbyProto.hasGroupbyNode()) { - distinctGroupby.setGroupbyPlan(convertGroupby(context, evalContext, nodeMap, + distinctGroupby.setGroupbyPlan(convertGroupby(evalContext, nodeMap, distinctGroupbyProto.getGroupbyNode())); } if (distinctGroupbyProto.getSubPlansCount() > 0) { List subPlans = TUtil.newList(); for (int i = 0; i < distinctGroupbyProto.getSubPlansCount(); i++) { - subPlans.add(convertGroupby(context, evalContext, nodeMap, distinctGroupbyProto.getSubPlans(i))); + subPlans.add(convertGroupby(evalContext, nodeMap, distinctGroupbyProto.getSubPlans(i))); } distinctGroupby.setSubPlans(subPlans); } @@ -336,11 +332,11 @@ private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf contex distinctGroupby.setGroupingColumns(convertColumns(distinctGroupbyProto.getGroupingKeysList())); } if (distinctGroupbyProto.getAggFunctionsCount() > 0) { - distinctGroupby.setAggFunctions(convertAggFuncCallEvals(context, evalContext, + distinctGroupby.setAggFunctions(convertAggFuncCallEvals(evalContext, distinctGroupbyProto.getAggFunctionsList())); } if (distinctGroupbyProto.getTargetsCount() > 0) { - distinctGroupby.setTargets(convertTargets(context, evalContext, distinctGroupbyProto.getTargetsList())); + distinctGroupby.setTargets(convertTargets(evalContext, distinctGroupbyProto.getTargetsList())); } int [] resultColumnIds = new int[distinctGroupbyProto.getResultIdCount()]; for (int i = 0; i < distinctGroupbyProto.getResultIdCount(); i++) { @@ -355,7 +351,7 @@ private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf contex return distinctGroupby; } - private static JoinNode convertJoin(OverridableConf context, EvalContext evalContext, + private static JoinNode convertJoin(EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.JoinNode joinProto = protoNode.getJoin(); @@ -366,16 +362,16 @@ private static JoinNode convertJoin(OverridableConf context, EvalContext evalCon join.setInSchema(convertSchema(protoNode.getInSchema())); join.setOutSchema(convertSchema(protoNode.getOutSchema())); if (joinProto.hasJoinQual()) { - join.setJoinQual(EvalNodeDeserializer.deserialize(context, evalContext, joinProto.getJoinQual())); + join.setJoinQual(EvalNodeDeserializer.deserialize(evalContext, joinProto.getJoinQual())); } if (joinProto.getExistsTargets()) { - join.setTargets(convertTargets(context, evalContext, joinProto.getTargetsList())); + join.setTargets(convertTargets(evalContext, joinProto.getTargetsList())); } return join; } - private static SelectionNode convertFilter(OverridableConf context, EvalContext evalContext, + private static SelectionNode convertFilter(EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.FilterNode filterProto = protoNode.getFilter(); @@ -383,7 +379,7 @@ private static SelectionNode convertFilter(OverridableConf context, EvalContext selection.setInSchema(convertSchema(protoNode.getInSchema())); selection.setOutSchema(convertSchema(protoNode.getOutSchema())); selection.setChild(nodeMap.get(filterProto.getChildSeq())); - selection.setQual(EvalNodeDeserializer.deserialize(context, evalContext, filterProto.getQual())); + selection.setQual(EvalNodeDeserializer.deserialize(evalContext, filterProto.getQual())); return selection; } @@ -400,14 +396,14 @@ private static UnionNode convertUnion(Map nodeMap, PlanPro return union; } - private static ScanNode convertScan(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode) { + private static ScanNode convertScan(EvalContext evalContext, PlanProto.LogicalNode protoNode) { ScanNode scan = new ScanNode(protoNode.getNodeId()); - fillScanNode(context, evalContext, protoNode, scan); + fillScanNode(evalContext, protoNode, scan); return scan; } - private static void fillScanNode(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode, + private static void fillScanNode(EvalContext evalContext, PlanProto.LogicalNode protoNode, ScanNode scan) { PlanProto.ScanNode scanProto = protoNode.getScan(); if (scanProto.hasAlias()) { @@ -417,11 +413,11 @@ private static void fillScanNode(OverridableConf context, EvalContext evalContex } if (scanProto.getExistTargets()) { - scan.setTargets(convertTargets(context, evalContext, scanProto.getTargetsList())); + scan.setTargets(convertTargets(evalContext, scanProto.getTargetsList())); } if (scanProto.hasQual()) { - scan.setQual(EvalNodeDeserializer.deserialize(context, evalContext, scanProto.getQual())); + scan.setQual(EvalNodeDeserializer.deserialize(evalContext, scanProto.getQual())); } if(scanProto.hasBroadcast()){ @@ -431,10 +427,10 @@ private static void fillScanNode(OverridableConf context, EvalContext evalContex scan.setOutSchema(convertSchema(protoNode.getOutSchema())); } - private static PartitionedTableScanNode convertPartitionScan(OverridableConf context, EvalContext evalContext, + private static PartitionedTableScanNode convertPartitionScan(EvalContext evalContext, PlanProto.LogicalNode protoNode) { PartitionedTableScanNode partitionedScan = new PartitionedTableScanNode(protoNode.getNodeId()); - fillScanNode(context, evalContext, protoNode, partitionedScan); + fillScanNode(evalContext, protoNode, partitionedScan); PlanProto.PartitionScanSpec partitionScanProto = protoNode.getPartitionScan(); Path [] paths = new Path[partitionScanProto.getPathsCount()]; @@ -445,7 +441,7 @@ private static PartitionedTableScanNode convertPartitionScan(OverridableConf con return partitionedScan; } - private static TableSubQueryNode convertTableSubQuery(OverridableConf context, EvalContext evalContext, + private static TableSubQueryNode convertTableSubQuery(EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.TableSubQueryNode proto = protoNode.getTableSubQuery(); @@ -454,7 +450,7 @@ private static TableSubQueryNode convertTableSubQuery(OverridableConf context, E tableSubQuery.init(proto.getTableName(), nodeMap.get(proto.getChildSeq())); tableSubQuery.setInSchema(convertSchema(protoNode.getInSchema())); if (proto.getTargetsCount() > 0) { - tableSubQuery.setTargets(convertTargets(context, evalContext, proto.getTargetsList())); + tableSubQuery.setTargets(convertTargets(evalContext, proto.getTargetsList())); } return tableSubQuery; @@ -611,21 +607,21 @@ private static TruncateTableNode convertTruncateTable(PlanProto.LogicalNode prot return truncateTable; } - private static AggregationFunctionCallEval [] convertAggFuncCallEvals(OverridableConf context, EvalContext evalContext, - List evalTrees) { + private static AggregationFunctionCallEval [] convertAggFuncCallEvals(EvalContext evalContext, + List evalTrees) { AggregationFunctionCallEval [] aggFuncs = new AggregationFunctionCallEval[evalTrees.size()]; for (int i = 0; i < aggFuncs.length; i++) { - aggFuncs[i] = (AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(context, evalContext, + aggFuncs[i] = (AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(evalContext, evalTrees.get(i)); } return aggFuncs; } - private static WindowFunctionEval[] convertWindowFunccEvals(OverridableConf context, EvalContext evalContext, + private static WindowFunctionEval[] convertWindowFunccEvals(EvalContext evalContext, List evalTrees) { WindowFunctionEval[] winFuncEvals = new WindowFunctionEval[evalTrees.size()]; for (int i = 0; i < winFuncEvals.length; i++) { - winFuncEvals[i] = (WindowFunctionEval) EvalNodeDeserializer.deserialize(context, evalContext, evalTrees.get(i)); + winFuncEvals[i] = (WindowFunctionEval) EvalNodeDeserializer.deserialize(evalContext, evalTrees.get(i)); } return winFuncEvals; } @@ -642,12 +638,12 @@ public static Column[] convertColumns(List columnProt return columns; } - public static Target[] convertTargets(OverridableConf context, EvalContext evalContext, + public static Target[] convertTargets(EvalContext evalContext, List targetsProto) { Target [] targets = new Target[targetsProto.size()]; for (int i = 0; i < targets.length; i++) { PlanProto.Target targetProto = targetsProto.get(i); - EvalNode evalNode = EvalNodeDeserializer.deserialize(context, evalContext, targetProto.getExpr()); + EvalNode evalNode = EvalNodeDeserializer.deserialize(evalContext, targetProto.getExpr()); if (targetProto.hasAlias()) { targets[i] = new Target(evalNode, targetProto.getAlias()); } else { From cedbb37cf675a4f1c134c9311733cf5d18a95963 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 17 Apr 2015 17:13:05 +0900 Subject: [PATCH 54/55] Revert "TAJO-1344_3" This reverts commit 43e687d36ab6a9961a274d78a91ff94f234cefc1. --- .../main/java/org/apache/tajo/QueryVars.java | 2 + .../java/org/apache/tajo/conf/TajoConf.java | 1 - .../tajo/engine/function/FunctionLoader.java | 6 +- .../rules/GlobalPlanEqualityTester.java | 2 +- .../tajo/engine/query/QueryContext.java | 8 ++ .../org/apache/tajo/master/GlobalEngine.java | 2 + .../org/apache/tajo/master/TajoMaster.java | 2 +- .../tajo/master/exec/QueryExecutor.java | 4 +- .../apache/tajo/util/QueryContextUtil.java | 28 +++++ .../org/apache/tajo/worker/TajoWorker.java | 2 +- .../java/org/apache/tajo/worker/Task.java | 9 +- .../tajo/worker/TaskAttemptContext.java | 3 +- .../apache/tajo/engine/eval/ExprTestBase.java | 12 +- .../org/apache/tajo/plan/ExprAnnotator.java | 6 +- .../tajo/plan/expr/BetweenPredicateEval.java | 3 +- .../org/apache/tajo/plan/expr/CastEval.java | 19 ++- .../apache/tajo/plan/expr/EvalContext.java | 11 -- .../org/apache/tajo/plan/expr/FieldEval.java | 3 +- .../apache/tajo/plan/expr/FunctionEval.java | 3 +- .../tajo/plan/expr/GeneralFunctionEval.java | 8 +- .../plan/expr/PatternMatchPredicateEval.java | 3 +- .../exprrewrite/rules/ConstantFolding.java | 4 +- .../function/python/PythonScriptEngine.java | 19 ++- .../function/python/TajoScriptEngine.java | 6 +- .../rules/LogicalPlanEqualityTester.java | 2 +- .../plan/serder/EvalNodeDeserializer.java | 6 +- .../plan/serder/LogicalNodeDeserializer.java | 108 +++++++++--------- 27 files changed, 153 insertions(+), 129 deletions(-) create mode 100644 tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java index 55ca7009f0..c3835f8bfb 100644 --- a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java +++ b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java @@ -29,6 +29,8 @@ public enum QueryVars implements ConfigKey { OUTPUT_OVERWRITE, OUTPUT_AS_DIRECTORY, OUTPUT_PER_FILE_SIZE, + PYTHON_SCRIPT_CODE_DIR, + PYTHON_CONTROLLER_LOG_DIR ; QueryVars() { diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index bfba290e9c..84ee05999c 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -295,7 +295,6 @@ public static enum ConfVars implements ConfigKey { // Python UDF PYTHON_CODE_DIR("tajo.function.python.code-dir", ""), - PYTHON_CONTROLLER_LOG_DIR("tajo.function.python.controller.log-dir", ""), ///////////////////////////////////////////////////////////////////////////////// // User Session Configuration diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index d9abc609be..c259e7b89f 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -84,15 +84,15 @@ public static Map load() { } /** - * Load functions that are user-defined functions. + * Load functions that are optionally defined by users. * * @param conf * @param functionMap * @return * @throws IOException */ - public static Map loadUserDefinedFunctions(TajoConf conf, - Map functionMap) + public static Map loadOptionalFunctions(TajoConf conf, + Map functionMap) throws IOException { String[] codePaths = conf.getStrings(TajoConf.ConfVars.PYTHON_CODE_DIR.varname); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java index d0c548f4c5..e55a258b4e 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/planner/global/rewriter/rules/GlobalPlanEqualityTester.java @@ -51,7 +51,7 @@ public MasterPlan rewrite(MasterPlan plan) { LogicalNode node = eb.getPlan(); if (node != null) { PlanProto.LogicalNodeTree tree = LogicalNodeSerializer.serialize(node); - LogicalNode deserialize = LogicalNodeDeserializer.deserialize(null, tree); + LogicalNode deserialize = LogicalNodeDeserializer.deserialize(plan.getContext(), null, tree); assert node.deepEquals(deserialize); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java index 7b3c00db30..0e595f79f3 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java @@ -151,6 +151,14 @@ public NodeType getCommandType() { return strVal != null ? NodeType.valueOf(strVal) : null; } + public void setPythonScriptPath(String path) { + put(QueryVars.PYTHON_SCRIPT_CODE_DIR, path); + } + + public String getPythonScriptPath() { + return get(QueryVars.PYTHON_SCRIPT_CODE_DIR); + } + public void setCreateTable() { setCommandType(NodeType.CREATE_TABLE); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java index 074f34e94c..04a0bad076 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java @@ -56,6 +56,7 @@ import org.apache.tajo.plan.verifier.VerifyException; import org.apache.tajo.storage.StorageManager; import org.apache.tajo.util.CommonTestingUtil; +import org.apache.tajo.util.QueryContextUtil; import java.io.IOException; import java.sql.SQLException; @@ -173,6 +174,7 @@ public Expr load(String sql) throws SQLSyntaxError { public SubmitQueryResponse executeQuery(Session session, String query, boolean isJson) { LOG.info("Query: " + query); QueryContext queryContext = createQueryContext(session); + QueryContextUtil.updatePythonScriptPath(context.getConf(), queryContext); Expr planningContext; try { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 0a5de58897..5d88ea0554 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -210,7 +210,7 @@ public void serviceInit(Configuration _conf) throws Exception { private Collection loadFunctions() throws IOException { Map functionMap = FunctionLoader.load(); - return FunctionLoader.loadUserDefinedFunctions(systemConf, functionMap).values(); + return FunctionLoader.loadOptionalFunctions(systemConf, functionMap).values(); } private void initSystemMetrics() { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index 90bb774a64..20a812f367 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -269,7 +269,7 @@ public void execNonFromQuery(QueryContext queryContext, LogicalPlan plan, Submit throws Exception { LogicalRootNode rootNode = plan.getRootBlock().getRoot(); - EvalContext evalContext = new EvalContext(queryContext); + EvalContext evalContext = new EvalContext(); Target[] targets = plan.getRootBlock().getRawTargets(); if (targets == null) { throw new PlanningException("No targets"); @@ -316,7 +316,7 @@ public static void startScriptExecutors(QueryContext queryContext, EvalContext e if (functionEval.getFuncDesc().getInvocation().hasPython()) { TajoScriptEngine scriptExecutor = new PythonScriptEngine(functionEval.getFuncDesc()); evalContext.addScriptEngine(eval, scriptExecutor); - scriptExecutor.start(queryContext.getConf()); + scriptExecutor.start(queryContext); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java new file mode 100644 index 0000000000..dbee78e5ba --- /dev/null +++ b/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.tajo.util; + +import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.engine.query.QueryContext; + +public class QueryContextUtil { + public static void updatePythonScriptPath(TajoConf systemConf, QueryContext queryContext) { + queryContext.setPythonScriptPath(systemConf.getVar(TajoConf.ConfVars.PYTHON_CODE_DIR)); + } +} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index 17af71a5a4..d66474516d 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -237,7 +237,7 @@ public void serviceInit(Configuration conf) throws Exception { historyReader = new HistoryReader(workerContext.getWorkerName(), this.systemConf); - FunctionLoader.loadUserDefinedFunctions(systemConf, new HashMap()); + FunctionLoader.loadOptionalFunctions(systemConf, new HashMap()); diagnoseTajoWorker(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index 76283d7022..db0833fa89 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -46,6 +46,7 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.*; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; import org.apache.tajo.master.cluster.WorkerConnectionInfo; +import org.apache.tajo.plan.function.python.PythonScriptEngine; import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.serder.LogicalNodeDeserializer; @@ -57,6 +58,7 @@ import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.NetUtils; +import org.apache.tajo.util.QueryContextUtil; import java.io.File; import java.io.IOException; @@ -100,6 +102,8 @@ public class Task { private Schema finalSchema = null; private TupleComparator sortComp = null; + private PythonScriptEngine pythonEngine; + public Task(String taskRunnerId, Path baseDir, TaskAttemptId taskId, @@ -134,7 +138,8 @@ public Task(String taskRunnerId, } public void initPlan() throws IOException { - plan = LogicalNodeDeserializer.deserialize(context.getEvalContext(), request.getPlan()); + QueryContextUtil.updatePythonScriptPath(systemConf, queryContext); + plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); if (scanNode != null) { for (LogicalNode node : scanNode) { @@ -191,7 +196,7 @@ public void initPlan() throws IOException { private void startScriptExecutors() throws IOException { for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.start(systemConf); + executor.start(queryContext); } } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java index 72eacd0c74..58028ac014 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TaskAttemptContext.java @@ -84,14 +84,13 @@ public class TaskAttemptContext { private Map partitionOutputVolume; private HashShuffleAppenderManager hashShuffleAppenderManager; - private EvalContext evalContext; + private EvalContext evalContext = new EvalContext(); public TaskAttemptContext(QueryContext queryContext, final ExecutionBlockContext executionBlockContext, final TaskAttemptId queryId, final FragmentProto[] fragments, final Path workDir) { this.queryContext = queryContext; - this.evalContext = new EvalContext(queryContext); if (executionBlockContext != null) { // For unit tests this.workerContext = executionBlockContext.getWorkerContext(); diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 8886bf23eb..9d6b1121a0 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -55,6 +55,7 @@ import org.apache.tajo.util.BytesUtils; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.KeyValueSet; +import org.apache.tajo.util.QueryContextUtil; import org.apache.tajo.util.datetime.DateTimeUtil; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -96,7 +97,7 @@ public static void setUp() throws Exception { cat.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); cat.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); Map map = FunctionLoader.load(); - map = FunctionLoader.loadUserDefinedFunctions(conf, map); + map = FunctionLoader.loadOptionalFunctions(conf, map); for (FunctionDesc funcDesc : map.values()) { cat.createFunction(funcDesc); } @@ -173,7 +174,7 @@ private static Target[] getRawTargets(QueryContext context, String query, boolea assertJsonSerDer(t.getEvalTree()); } for (Target t : targets) { - assertEvalTreeProtoSerDer(t.getEvalTree()); + assertEvalTreeProtoSerDer(context, t.getEvalTree()); } return targets; } @@ -225,6 +226,7 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S queryContext = LocalTajoTestingUtility.createDummyContext(conf); queryContext.putAll(context); } + QueryContextUtil.updatePythonScriptPath(conf, queryContext); String timezoneId = queryContext.get(SessionVars.TIMEZONE); TimeZone timeZone = TimeZone.getTimeZone(timezoneId); @@ -271,7 +273,7 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S Target [] targets; TajoClassLoader classLoader = new TajoClassLoader(); - EvalContext evalContext = new EvalContext(queryContext); + EvalContext evalContext = new EvalContext(); try { targets = getRawTargets(queryContext, query, condition); @@ -330,8 +332,8 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S } } - public static void assertEvalTreeProtoSerDer(EvalNode evalNode) { + public static void assertEvalTreeProtoSerDer(OverridableConf context, EvalNode evalNode) { PlanProto.EvalNodeTree converted = EvalNodeSerializer.serialize(evalNode); - assertEquals(evalNode, EvalNodeDeserializer.deserialize(null, converted)); + assertEquals(evalNode, EvalNodeDeserializer.deserialize(context, null, converted)); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java index 564975816d..0c5a01263d 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/ExprAnnotator.java @@ -188,7 +188,7 @@ private static EvalNode convertType(Context ctx, EvalNode evalNode, DataType toT return newConst; } else { - return new CastEval(evalNode, toType); + return new CastEval(ctx.queryContext, evalNode, toType); } } @@ -609,7 +609,7 @@ public EvalNode visitFunction(Context ctx, Stack stack, FunctionExpr expr) FunctionType functionType = funcDesc.getFuncType(); if (functionType == FunctionType.GENERAL || functionType == FunctionType.UDF) { - return new GeneralFunctionEval(funcDesc, givenArgs); + return new GeneralFunctionEval(ctx.queryContext, funcDesc, givenArgs); } else if (functionType == FunctionType.AGGREGATION || functionType == FunctionType.UDA) { if (!ctx.currentBlock.hasNode(NodeType.GROUP_BY)) { @@ -801,7 +801,7 @@ public EvalNode visitCastExpr(Context ctx, Stack stack, CastExpr expr) thr LogicalPlanner.convertDataType(expr.getTarget()).getDataType(), tz)); } else { - return new CastEval(child, LogicalPlanner.convertDataType(expr.getTarget()).getDataType()); + return new CastEval(ctx.queryContext, child, LogicalPlanner.convertDataType(expr.getTarget()).getDataType()); } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java index 6002c185e7..eb3e36bd2f 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/BetweenPredicateEval.java @@ -20,7 +20,6 @@ import com.google.gson.annotations.Expose; -import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes; @@ -228,7 +227,7 @@ public String toString() { } @Override - public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { + public EvalNode bind(EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); if (begin.getType() == EvalType.CONST && end.getType() == EvalType.CONST) { Datum beginValue = ((ConstEval)begin).getValue(); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java index b0cce3c094..b8b768bd0c 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/CastEval.java @@ -20,9 +20,8 @@ import com.google.gson.annotations.Expose; +import org.apache.tajo.OverridableConf; import org.apache.tajo.SessionVars; -import org.apache.tajo.annotation.Nullable; -import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; import org.apache.tajo.datum.DatumFactory; import org.apache.tajo.storage.Tuple; @@ -36,9 +35,14 @@ public class CastEval extends UnaryEval implements Cloneable { @Expose private DataType target; @Expose private TimeZone timezone; - public CastEval(EvalNode operand, DataType target) { + public CastEval(OverridableConf context, EvalNode operand, DataType target) { super(EvalType.CAST, operand); this.target = target; + + if (context.containsKey(SessionVars.TIMEZONE)) { + String timezoneId = context.get(SessionVars.TIMEZONE); + timezone = TimeZone.getTimeZone(timezoneId); + } } public EvalNode getOperand() { @@ -63,15 +67,6 @@ public String getName() { return target.getType().name(); } - public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { - super.bind(evalContext, schema); - if (evalContext.getQueryContext().containsKey(SessionVars.TIMEZONE)) { - String timezoneId = evalContext.getQueryContext().get(SessionVars.TIMEZONE); - timezone = TimeZone.getTimeZone(timezoneId); - } - return this; - } - @Override @SuppressWarnings("unchecked") public Datum eval(Tuple tuple) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java index 928303bfd0..6a30e772ce 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/EvalContext.java @@ -18,7 +18,6 @@ package org.apache.tajo.plan.expr; -import org.apache.tajo.OverridableConf; import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.util.TUtil; @@ -26,14 +25,8 @@ import java.util.Map; public class EvalContext { - private final OverridableConf queryContext; - private final Map scriptEngineMap = TUtil.newHashMap(); - public EvalContext(OverridableConf queryContext) { - this.queryContext = queryContext; - } - public void addScriptEngine(EvalNode evalNode, TajoScriptEngine scriptExecutor) { this.scriptEngineMap.put(evalNode, scriptExecutor); } @@ -49,8 +42,4 @@ public TajoScriptEngine getScriptEngine(EvalNode evalNode) { public Collection getAllScriptEngines() { return this.scriptEngineMap.values(); } - - public OverridableConf getQueryContext() { - return queryContext; - } } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java index 15f916e743..870970b538 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FieldEval.java @@ -19,7 +19,6 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; -import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Column; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes.DataType; @@ -41,7 +40,7 @@ public FieldEval(Column column) { } @Override - public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { + public EvalNode bind(EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); // TODO - column namespace should be improved to simplify name handling and resolving. if (column.hasQualifier()) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java index 1dc26e56e1..dd9121bd5a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/FunctionEval.java @@ -21,7 +21,6 @@ import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.gson.annotations.Expose; -import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes; @@ -50,7 +49,7 @@ public FunctionEval(EvalType type, FunctionDesc funcDesc, EvalNode[] argEvals) { } @Override - public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { + public EvalNode bind(EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); this.params = new VTuple(argEvals.length); return this; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java index 81a4fcc54a..30fbe910f5 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/GeneralFunctionEval.java @@ -20,7 +20,6 @@ import com.google.gson.annotations.Expose; import org.apache.tajo.OverridableConf; -import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.FunctionDesc; import org.apache.tajo.catalog.Schema; import org.apache.tajo.datum.Datum; @@ -34,16 +33,15 @@ public class GeneralFunctionEval extends FunctionEval { protected FunctionInvoke funcInvoke; @Expose protected FunctionInvokeContext invokeContext; - public GeneralFunctionEval(FunctionDesc desc, EvalNode[] givenArgs) + public GeneralFunctionEval(OverridableConf queryContext, FunctionDesc desc, EvalNode[] givenArgs) throws IOException { super(EvalType.FUNCTION, desc, givenArgs); + this.invokeContext = new FunctionInvokeContext(queryContext, getParamType()); } @Override - public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { + public EvalNode bind(EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); - - this.invokeContext = new FunctionInvokeContext(evalContext.getQueryContext(), getParamType()); try { this.funcInvoke = FunctionInvoke.newInstance(funcDesc); if (evalContext != null && evalContext.hasScriptEngine(this)) { diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java index 871c2ab66c..cdd8dfb575 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/expr/PatternMatchPredicateEval.java @@ -19,7 +19,6 @@ package org.apache.tajo.plan.expr; import com.google.gson.annotations.Expose; -import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.CatalogUtil; import org.apache.tajo.catalog.Schema; import org.apache.tajo.common.TajoDataTypes; @@ -75,7 +74,7 @@ public String getName() { } @Override - public EvalNode bind(@Nullable EvalContext evalContext, Schema schema) { + public EvalNode bind(EvalContext evalContext, Schema schema) { super.bind(evalContext, schema); compile(pattern); return this; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index 5e88ae4f0a..942c3b47cd 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -94,8 +94,8 @@ public EvalNode visitFuncCall(LogicalPlanner.PlanContext context, FunctionEval e if (evalNode.getFuncDesc().getInvocation().hasPython()) { TajoScriptEngine executor = new PythonScriptEngine(evalNode.getFuncDesc()); try { - executor.start(context.getQueryContext().getConf()); - EvalContext evalContext = new EvalContext(context.getQueryContext()); + executor.start(context.getQueryContext()); + EvalContext evalContext = new EvalContext(); evalContext.addScriptEngine(evalNode, executor); evalNode.bind(evalContext, null); Datum funcRes = evalNode.eval(null); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index 15eac6b2b6..d081665fa8 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -21,7 +21,6 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; import org.apache.tajo.OverridableConf; import org.apache.tajo.QueryVars; import org.apache.tajo.catalog.*; @@ -171,7 +170,7 @@ private static List getFunctions(InputStream is) throws IOException { private static final int CONTROLLER_LOG_FILE_PATH = 8; // Controller log file logs progress through the controller script not user code. private static final int OUT_SCHEMA = 9; // the schema of the output column - private Configuration systemConf; + private OverridableConf queryContext; private Process process; // Handle to the external execution of python functions @@ -208,9 +207,8 @@ public PythonScriptEngine(FunctionDesc functionDesc) { outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); } - @Override - public void start(Configuration systemConf) throws IOException { - this.systemConf = systemConf; + public void start(OverridableConf queryContext) throws IOException { + this.queryContext = queryContext; startUdfController(); createInputHandlers(); setStreams(); @@ -219,7 +217,6 @@ public void start(Configuration systemConf) throws IOException { } } - @Override public void shutdown() { process.destroy(); FileUtil.cleanup(LOG, stdin); @@ -250,10 +247,10 @@ private String[] buildCommand() throws IOException { String[] command = new String[10]; // TODO: support controller logging - String standardOutputRootWriteLocation = systemConf.get(TajoConf.ConfVars.PYTHON_CONTROLLER_LOG_DIR.keyname(), - DEFAULT_LOG_DIR); - if (standardOutputRootWriteLocation != null) { + String standardOutputRootWriteLocation = ""; + if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { LOG.warn("Currently, logging is not supported for the python controller."); + standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR, DEFAULT_LOG_DIR); } String controllerLogFileName, outFileName, errOutFileName; @@ -272,10 +269,10 @@ private String[] buildCommand() throws IOException { command[UDF_FILE_NAME] = fileName; command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; - String fileCachePath = systemConf.get(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname()); - if (fileCachePath == null) { + if (!queryContext.containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); } + String fileCachePath = queryContext.get(QueryVars.PYTHON_SCRIPT_CODE_DIR); command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; command[STD_OUT_OUTPUT_PATH] = outFileName; command[STD_ERR_OUTPUT_PATH] = errOutFileName; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 726ec2faf7..2ba9c2b4d2 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -18,7 +18,7 @@ package org.apache.tajo.plan.function.python; -import org.apache.hadoop.conf.Configuration; +import org.apache.tajo.OverridableConf; import org.apache.tajo.datum.Datum; import org.apache.tajo.storage.Tuple; @@ -62,10 +62,10 @@ protected static InputStream getScriptAsStream(URI scriptPath) { /** * Start TajoScriptEngine. * - * @param systemConf + * @param queryContext * @throws IOException */ - public abstract void start(Configuration systemConf) throws IOException; + public abstract void start(OverridableConf queryContext) throws IOException; /** * Shutdown TajoScriptEngine. diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java index 7ca4f59722..43a86184b3 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/rewrite/rules/LogicalPlanEqualityTester.java @@ -48,7 +48,7 @@ public boolean isEligible(OverridableConf queryContext, LogicalPlan plan) { public LogicalPlan rewrite(OverridableConf queryContext, LogicalPlan plan) throws PlanningException { LogicalNode root = plan.getRootBlock().getRoot(); PlanProto.LogicalNodeTree serialized = LogicalNodeSerializer.serialize(plan.getRootBlock().getRoot()); - LogicalNode deserialized = LogicalNodeDeserializer.deserialize(null, serialized); + LogicalNode deserialized = LogicalNodeDeserializer.deserialize(queryContext, null, serialized); assert root.deepEquals(deserialized); return plan; } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java index dae56d4332..3ca76ee01a 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/EvalNodeDeserializer.java @@ -53,7 +53,7 @@ */ public class EvalNodeDeserializer { - public static EvalNode deserialize(EvalContext evalContext, PlanProto.EvalNodeTree tree) { + public static EvalNode deserialize(OverridableConf context, EvalContext evalContext, PlanProto.EvalNodeTree tree) { Map evalNodeMap = Maps.newHashMap(); // sort serialized eval nodes in an ascending order of their IDs. @@ -89,7 +89,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { current = new IsNullEval(unaryProto.getNegative(), child); break; case CAST: - current = new CastEval(child, unaryProto.getCastingType()); + current = new CastEval(context, child, unaryProto.getCastingType()); break; case SIGNED: current = new SignedEval(unaryProto.getNegative(), child); @@ -181,7 +181,7 @@ public int compare(PlanProto.EvalNode o1, PlanProto.EvalNode o2) { try { funcDesc = new FunctionDesc(funcProto.getFuncion()); if (type == EvalType.FUNCTION) { - current = new GeneralFunctionEval(funcDesc, params); + current = new GeneralFunctionEval(context, funcDesc, params); if (evalContext != null && funcDesc.getInvocation().hasPython()) { evalContext.addScriptEngine(current, new PythonScriptEngine(funcDesc)); } diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java index 3101ca1189..84991bbf75 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/serder/LogicalNodeDeserializer.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.hadoop.fs.Path; +import org.apache.tajo.OverridableConf; import org.apache.tajo.algebra.JoinType; import org.apache.tajo.annotation.Nullable; import org.apache.tajo.catalog.Column; @@ -51,11 +52,12 @@ public class LogicalNodeDeserializer { /** * Deserialize a list of nodes into a logical node tree. * - * @param evalContext eval context + * @param context QueryContext * @param tree LogicalNodeTree which contains a list of serialized logical nodes. * @return A logical node tree */ - public static LogicalNode deserialize(@Nullable EvalContext evalContext, PlanProto.LogicalNodeTree tree) { + public static LogicalNode deserialize(OverridableConf context, @Nullable EvalContext evalContext, + PlanProto.LogicalNodeTree tree) { Map nodeMap = Maps.newHashMap(); // sort serialized logical nodes in an ascending order of their sids @@ -85,10 +87,10 @@ public int compare(PlanProto.LogicalNode o1, PlanProto.LogicalNode o2) { current = convertSetSession(protoNode); break; case EXPRS: - current = convertEvalExpr(evalContext, protoNode); + current = convertEvalExpr(context, evalContext, protoNode); break; case PROJECTION: - current = convertProjection(evalContext, nodeMap, protoNode); + current = convertProjection(context, evalContext, nodeMap, protoNode); break; case LIMIT: current = convertLimit(nodeMap, protoNode); @@ -97,34 +99,34 @@ public int compare(PlanProto.LogicalNode o1, PlanProto.LogicalNode o2) { current = convertSort(nodeMap, protoNode); break; case WINDOW_AGG: - current = convertWindowAgg(evalContext, nodeMap, protoNode); + current = convertWindowAgg(context, evalContext, nodeMap, protoNode); break; case HAVING: - current = convertHaving(evalContext, nodeMap, protoNode); + current = convertHaving(context, evalContext, nodeMap, protoNode); break; case GROUP_BY: - current = convertGroupby(evalContext, nodeMap, protoNode); + current = convertGroupby(context, evalContext, nodeMap, protoNode); break; case DISTINCT_GROUP_BY: - current = convertDistinctGroupby(evalContext, nodeMap, protoNode); + current = convertDistinctGroupby(context, evalContext, nodeMap, protoNode); break; case SELECTION: - current = convertFilter(evalContext, nodeMap, protoNode); + current = convertFilter(context, evalContext, nodeMap, protoNode); break; case JOIN: - current = convertJoin(evalContext, nodeMap, protoNode); + current = convertJoin(context, evalContext, nodeMap, protoNode); break; case TABLE_SUBQUERY: - current = convertTableSubQuery(evalContext, nodeMap, protoNode); + current = convertTableSubQuery(context, evalContext, nodeMap, protoNode); break; case UNION: current = convertUnion(nodeMap, protoNode); break; case PARTITIONS_SCAN: - current = convertPartitionScan(evalContext, protoNode); + current = convertPartitionScan(context, evalContext, protoNode); break; case SCAN: - current = convertScan(evalContext, protoNode); + current = convertScan(context, evalContext, protoNode); break; case CREATE_TABLE: @@ -189,24 +191,24 @@ private static SetSessionNode convertSetSession(PlanProto.LogicalNode protoNode) return setSession; } - private static EvalExprNode convertEvalExpr(EvalContext evalContext, + private static EvalExprNode convertEvalExpr(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode) { PlanProto.EvalExprNode evalExprProto = protoNode.getExprEval(); EvalExprNode evalExpr = new EvalExprNode(protoNode.getNodeId()); evalExpr.setInSchema(convertSchema(protoNode.getInSchema())); - evalExpr.setTargets(convertTargets(evalContext, evalExprProto.getTargetsList())); + evalExpr.setTargets(convertTargets(context, evalContext, evalExprProto.getTargetsList())); return evalExpr; } - private static ProjectionNode convertProjection(EvalContext evalContext, + private static ProjectionNode convertProjection(OverridableConf context, EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.ProjectionNode projectionProto = protoNode.getProjection(); ProjectionNode projectionNode = new ProjectionNode(protoNode.getNodeId()); - projectionNode.init(projectionProto.getDistinct(), convertTargets(evalContext, + projectionNode.init(projectionProto.getDistinct(), convertTargets(context, evalContext, projectionProto.getTargetsList())); projectionNode.setChild(nodeMap.get(projectionProto.getChildSeq())); projectionNode.setInSchema(convertSchema(protoNode.getInSchema())); @@ -239,20 +241,21 @@ private static SortNode convertSort(Map nodeMap, PlanProto return sortNode; } - private static HavingNode convertHaving(EvalContext evalContext, + private static HavingNode convertHaving(OverridableConf context, EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.FilterNode havingProto = protoNode.getFilter(); HavingNode having = new HavingNode(protoNode.getNodeId()); having.setChild(nodeMap.get(havingProto.getChildSeq())); - having.setQual(EvalNodeDeserializer.deserialize(evalContext, havingProto.getQual())); + having.setQual(EvalNodeDeserializer.deserialize(context, evalContext, havingProto.getQual())); having.setInSchema(convertSchema(protoNode.getInSchema())); having.setOutSchema(convertSchema(protoNode.getOutSchema())); return having; } - private static WindowAggNode convertWindowAgg(EvalContext evalContext, Map nodeMap, + private static WindowAggNode convertWindowAgg(OverridableConf context, EvalContext evalContext, + Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.WindowAggNode windowAggProto = protoNode.getWindowAgg(); @@ -264,7 +267,7 @@ private static WindowAggNode convertWindowAgg(EvalContext evalContext, Map 0) { - windowAgg.setWindowFunctions(convertWindowFunccEvals(evalContext, + windowAgg.setWindowFunctions(convertWindowFunccEvals(context, evalContext, windowAggProto.getWindowFunctionsList())); } @@ -275,7 +278,7 @@ private static WindowAggNode convertWindowAgg(EvalContext evalContext, Map 0) { - windowAgg.setTargets(convertTargets(evalContext, windowAggProto.getTargetsList())); + windowAgg.setTargets(convertTargets(context, evalContext, windowAggProto.getTargetsList())); } windowAgg.setInSchema(convertSchema(protoNode.getInSchema())); @@ -284,8 +287,8 @@ private static WindowAggNode convertWindowAgg(EvalContext evalContext, Map nodeMap, - PlanProto.LogicalNode protoNode) { + private static GroupbyNode convertGroupby(OverridableConf context, EvalContext evalContext, + Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.GroupbyNode groupbyProto = protoNode.getGroupby(); GroupbyNode groupby = new GroupbyNode(protoNode.getNodeId()); @@ -296,10 +299,10 @@ private static GroupbyNode convertGroupby(EvalContext evalContext, Map 0) { - groupby.setAggFunctions(convertAggFuncCallEvals(evalContext, groupbyProto.getAggFunctionsList())); + groupby.setAggFunctions(convertAggFuncCallEvals(context, evalContext, groupbyProto.getAggFunctionsList())); } if (groupbyProto.getTargetsCount() > 0) { - groupby.setTargets(convertTargets(evalContext, groupbyProto.getTargetsList())); + groupby.setTargets(convertTargets(context, evalContext, groupbyProto.getTargetsList())); } groupby.setInSchema(convertSchema(protoNode.getInSchema())); @@ -308,7 +311,8 @@ private static GroupbyNode convertGroupby(EvalContext evalContext, Map nodeMap, + private static DistinctGroupbyNode convertDistinctGroupby(OverridableConf context, EvalContext evalContext, + Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.DistinctGroupbyNode distinctGroupbyProto = protoNode.getDistinctGroupby(); @@ -316,14 +320,14 @@ private static DistinctGroupbyNode convertDistinctGroupby(EvalContext evalContex distinctGroupby.setChild(nodeMap.get(distinctGroupbyProto.getChildSeq())); if (distinctGroupbyProto.hasGroupbyNode()) { - distinctGroupby.setGroupbyPlan(convertGroupby(evalContext, nodeMap, + distinctGroupby.setGroupbyPlan(convertGroupby(context, evalContext, nodeMap, distinctGroupbyProto.getGroupbyNode())); } if (distinctGroupbyProto.getSubPlansCount() > 0) { List subPlans = TUtil.newList(); for (int i = 0; i < distinctGroupbyProto.getSubPlansCount(); i++) { - subPlans.add(convertGroupby(evalContext, nodeMap, distinctGroupbyProto.getSubPlans(i))); + subPlans.add(convertGroupby(context, evalContext, nodeMap, distinctGroupbyProto.getSubPlans(i))); } distinctGroupby.setSubPlans(subPlans); } @@ -332,11 +336,11 @@ private static DistinctGroupbyNode convertDistinctGroupby(EvalContext evalContex distinctGroupby.setGroupingColumns(convertColumns(distinctGroupbyProto.getGroupingKeysList())); } if (distinctGroupbyProto.getAggFunctionsCount() > 0) { - distinctGroupby.setAggFunctions(convertAggFuncCallEvals(evalContext, + distinctGroupby.setAggFunctions(convertAggFuncCallEvals(context, evalContext, distinctGroupbyProto.getAggFunctionsList())); } if (distinctGroupbyProto.getTargetsCount() > 0) { - distinctGroupby.setTargets(convertTargets(evalContext, distinctGroupbyProto.getTargetsList())); + distinctGroupby.setTargets(convertTargets(context, evalContext, distinctGroupbyProto.getTargetsList())); } int [] resultColumnIds = new int[distinctGroupbyProto.getResultIdCount()]; for (int i = 0; i < distinctGroupbyProto.getResultIdCount(); i++) { @@ -351,7 +355,7 @@ private static DistinctGroupbyNode convertDistinctGroupby(EvalContext evalContex return distinctGroupby; } - private static JoinNode convertJoin(EvalContext evalContext, + private static JoinNode convertJoin(OverridableConf context, EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.JoinNode joinProto = protoNode.getJoin(); @@ -362,16 +366,16 @@ private static JoinNode convertJoin(EvalContext evalContext, join.setInSchema(convertSchema(protoNode.getInSchema())); join.setOutSchema(convertSchema(protoNode.getOutSchema())); if (joinProto.hasJoinQual()) { - join.setJoinQual(EvalNodeDeserializer.deserialize(evalContext, joinProto.getJoinQual())); + join.setJoinQual(EvalNodeDeserializer.deserialize(context, evalContext, joinProto.getJoinQual())); } if (joinProto.getExistsTargets()) { - join.setTargets(convertTargets(evalContext, joinProto.getTargetsList())); + join.setTargets(convertTargets(context, evalContext, joinProto.getTargetsList())); } return join; } - private static SelectionNode convertFilter(EvalContext evalContext, + private static SelectionNode convertFilter(OverridableConf context, EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.FilterNode filterProto = protoNode.getFilter(); @@ -379,7 +383,7 @@ private static SelectionNode convertFilter(EvalContext evalContext, selection.setInSchema(convertSchema(protoNode.getInSchema())); selection.setOutSchema(convertSchema(protoNode.getOutSchema())); selection.setChild(nodeMap.get(filterProto.getChildSeq())); - selection.setQual(EvalNodeDeserializer.deserialize(evalContext, filterProto.getQual())); + selection.setQual(EvalNodeDeserializer.deserialize(context, evalContext, filterProto.getQual())); return selection; } @@ -396,14 +400,14 @@ private static UnionNode convertUnion(Map nodeMap, PlanPro return union; } - private static ScanNode convertScan(EvalContext evalContext, PlanProto.LogicalNode protoNode) { + private static ScanNode convertScan(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode) { ScanNode scan = new ScanNode(protoNode.getNodeId()); - fillScanNode(evalContext, protoNode, scan); + fillScanNode(context, evalContext, protoNode, scan); return scan; } - private static void fillScanNode(EvalContext evalContext, PlanProto.LogicalNode protoNode, + private static void fillScanNode(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode, ScanNode scan) { PlanProto.ScanNode scanProto = protoNode.getScan(); if (scanProto.hasAlias()) { @@ -413,11 +417,11 @@ private static void fillScanNode(EvalContext evalContext, PlanProto.LogicalNode } if (scanProto.getExistTargets()) { - scan.setTargets(convertTargets(evalContext, scanProto.getTargetsList())); + scan.setTargets(convertTargets(context, evalContext, scanProto.getTargetsList())); } if (scanProto.hasQual()) { - scan.setQual(EvalNodeDeserializer.deserialize(evalContext, scanProto.getQual())); + scan.setQual(EvalNodeDeserializer.deserialize(context, evalContext, scanProto.getQual())); } if(scanProto.hasBroadcast()){ @@ -427,10 +431,10 @@ private static void fillScanNode(EvalContext evalContext, PlanProto.LogicalNode scan.setOutSchema(convertSchema(protoNode.getOutSchema())); } - private static PartitionedTableScanNode convertPartitionScan(EvalContext evalContext, + private static PartitionedTableScanNode convertPartitionScan(OverridableConf context, EvalContext evalContext, PlanProto.LogicalNode protoNode) { PartitionedTableScanNode partitionedScan = new PartitionedTableScanNode(protoNode.getNodeId()); - fillScanNode(evalContext, protoNode, partitionedScan); + fillScanNode(context, evalContext, protoNode, partitionedScan); PlanProto.PartitionScanSpec partitionScanProto = protoNode.getPartitionScan(); Path [] paths = new Path[partitionScanProto.getPathsCount()]; @@ -441,7 +445,7 @@ private static PartitionedTableScanNode convertPartitionScan(EvalContext evalCon return partitionedScan; } - private static TableSubQueryNode convertTableSubQuery(EvalContext evalContext, + private static TableSubQueryNode convertTableSubQuery(OverridableConf context, EvalContext evalContext, Map nodeMap, PlanProto.LogicalNode protoNode) { PlanProto.TableSubQueryNode proto = protoNode.getTableSubQuery(); @@ -450,7 +454,7 @@ private static TableSubQueryNode convertTableSubQuery(EvalContext evalContext, tableSubQuery.init(proto.getTableName(), nodeMap.get(proto.getChildSeq())); tableSubQuery.setInSchema(convertSchema(protoNode.getInSchema())); if (proto.getTargetsCount() > 0) { - tableSubQuery.setTargets(convertTargets(evalContext, proto.getTargetsList())); + tableSubQuery.setTargets(convertTargets(context, evalContext, proto.getTargetsList())); } return tableSubQuery; @@ -607,21 +611,21 @@ private static TruncateTableNode convertTruncateTable(PlanProto.LogicalNode prot return truncateTable; } - private static AggregationFunctionCallEval [] convertAggFuncCallEvals(EvalContext evalContext, - List evalTrees) { + private static AggregationFunctionCallEval [] convertAggFuncCallEvals(OverridableConf context, EvalContext evalContext, + List evalTrees) { AggregationFunctionCallEval [] aggFuncs = new AggregationFunctionCallEval[evalTrees.size()]; for (int i = 0; i < aggFuncs.length; i++) { - aggFuncs[i] = (AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(evalContext, + aggFuncs[i] = (AggregationFunctionCallEval) EvalNodeDeserializer.deserialize(context, evalContext, evalTrees.get(i)); } return aggFuncs; } - private static WindowFunctionEval[] convertWindowFunccEvals(EvalContext evalContext, + private static WindowFunctionEval[] convertWindowFunccEvals(OverridableConf context, EvalContext evalContext, List evalTrees) { WindowFunctionEval[] winFuncEvals = new WindowFunctionEval[evalTrees.size()]; for (int i = 0; i < winFuncEvals.length; i++) { - winFuncEvals[i] = (WindowFunctionEval) EvalNodeDeserializer.deserialize(evalContext, evalTrees.get(i)); + winFuncEvals[i] = (WindowFunctionEval) EvalNodeDeserializer.deserialize(context, evalContext, evalTrees.get(i)); } return winFuncEvals; } @@ -638,12 +642,12 @@ public static Column[] convertColumns(List columnProt return columns; } - public static Target[] convertTargets(EvalContext evalContext, + public static Target[] convertTargets(OverridableConf context, EvalContext evalContext, List targetsProto) { Target [] targets = new Target[targetsProto.size()]; for (int i = 0; i < targets.length; i++) { PlanProto.Target targetProto = targetsProto.get(i); - EvalNode evalNode = EvalNodeDeserializer.deserialize(evalContext, targetProto.getExpr()); + EvalNode evalNode = EvalNodeDeserializer.deserialize(context, evalContext, targetProto.getExpr()); if (targetProto.hasAlias()) { targets[i] = new Target(evalNode, targetProto.getAlias()); } else { From f25aebd028a5589b78c13c18ca02c7beaca2bfa4 Mon Sep 17 00:00:00 2001 From: Jihoon Son Date: Fri, 17 Apr 2015 17:23:53 +0900 Subject: [PATCH 55/55] TAJO-1344_3 --- .../main/java/org/apache/tajo/QueryVars.java | 2 -- .../java/org/apache/tajo/conf/TajoConf.java | 1 + .../tajo/engine/function/FunctionLoader.java | 6 ++-- .../tajo/engine/query/QueryContext.java | 10 +------ .../org/apache/tajo/master/GlobalEngine.java | 2 -- .../org/apache/tajo/master/TajoMaster.java | 2 +- .../tajo/master/exec/QueryExecutor.java | 2 +- .../apache/tajo/util/QueryContextUtil.java | 28 ------------------- .../org/apache/tajo/worker/TajoWorker.java | 2 +- .../java/org/apache/tajo/worker/Task.java | 7 +---- .../apache/tajo/engine/eval/ExprTestBase.java | 4 +-- .../exprrewrite/rules/ConstantFolding.java | 2 +- .../function/python/PythonScriptEngine.java | 21 +++++++------- .../function/python/TajoScriptEngine.java | 6 ++-- 14 files changed, 25 insertions(+), 70 deletions(-) delete mode 100644 tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java diff --git a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java index c3835f8bfb..55ca7009f0 100644 --- a/tajo-common/src/main/java/org/apache/tajo/QueryVars.java +++ b/tajo-common/src/main/java/org/apache/tajo/QueryVars.java @@ -29,8 +29,6 @@ public enum QueryVars implements ConfigKey { OUTPUT_OVERWRITE, OUTPUT_AS_DIRECTORY, OUTPUT_PER_FILE_SIZE, - PYTHON_SCRIPT_CODE_DIR, - PYTHON_CONTROLLER_LOG_DIR ; QueryVars() { diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 84ee05999c..bfba290e9c 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -295,6 +295,7 @@ public static enum ConfVars implements ConfigKey { // Python UDF PYTHON_CODE_DIR("tajo.function.python.code-dir", ""), + PYTHON_CONTROLLER_LOG_DIR("tajo.function.python.controller.log-dir", ""), ///////////////////////////////////////////////////////////////////////////////// // User Session Configuration diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java index c259e7b89f..6061d1b233 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/function/FunctionLoader.java @@ -84,15 +84,15 @@ public static Map load() { } /** - * Load functions that are optionally defined by users. + * Load functions that are defined by users. * * @param conf * @param functionMap * @return * @throws IOException */ - public static Map loadOptionalFunctions(TajoConf conf, - Map functionMap) + public static Map loadUserDefinedFunctions(TajoConf conf, + Map functionMap) throws IOException { String[] codePaths = conf.getStrings(TajoConf.ConfVars.PYTHON_CODE_DIR.varname); diff --git a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java index 0e595f79f3..ee50221764 100644 --- a/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java +++ b/tajo-core/src/main/java/org/apache/tajo/engine/query/QueryContext.java @@ -25,8 +25,8 @@ import org.apache.tajo.SessionVars; import org.apache.tajo.catalog.partition.PartitionMethodDesc; import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.session.Session; import org.apache.tajo.plan.logical.NodeType; +import org.apache.tajo.session.Session; import static org.apache.tajo.rpc.protocolrecords.PrimitiveProtos.KeyValueSetProto; @@ -151,14 +151,6 @@ public NodeType getCommandType() { return strVal != null ? NodeType.valueOf(strVal) : null; } - public void setPythonScriptPath(String path) { - put(QueryVars.PYTHON_SCRIPT_CODE_DIR, path); - } - - public String getPythonScriptPath() { - return get(QueryVars.PYTHON_SCRIPT_CODE_DIR); - } - public void setCreateTable() { setCommandType(NodeType.CREATE_TABLE); } diff --git a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java index 04a0bad076..074f34e94c 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/GlobalEngine.java @@ -56,7 +56,6 @@ import org.apache.tajo.plan.verifier.VerifyException; import org.apache.tajo.storage.StorageManager; import org.apache.tajo.util.CommonTestingUtil; -import org.apache.tajo.util.QueryContextUtil; import java.io.IOException; import java.sql.SQLException; @@ -174,7 +173,6 @@ public Expr load(String sql) throws SQLSyntaxError { public SubmitQueryResponse executeQuery(Session session, String query, boolean isJson) { LOG.info("Query: " + query); QueryContext queryContext = createQueryContext(session); - QueryContextUtil.updatePythonScriptPath(context.getConf(), queryContext); Expr planningContext; try { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java index 5d88ea0554..0a5de58897 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/TajoMaster.java @@ -210,7 +210,7 @@ public void serviceInit(Configuration _conf) throws Exception { private Collection loadFunctions() throws IOException { Map functionMap = FunctionLoader.load(); - return FunctionLoader.loadOptionalFunctions(systemConf, functionMap).values(); + return FunctionLoader.loadUserDefinedFunctions(systemConf, functionMap).values(); } private void initSystemMetrics() { diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java index 20a812f367..ad1a8e35d8 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/QueryExecutor.java @@ -316,7 +316,7 @@ public static void startScriptExecutors(QueryContext queryContext, EvalContext e if (functionEval.getFuncDesc().getInvocation().hasPython()) { TajoScriptEngine scriptExecutor = new PythonScriptEngine(functionEval.getFuncDesc()); evalContext.addScriptEngine(eval, scriptExecutor); - scriptExecutor.start(queryContext); + scriptExecutor.start(queryContext.getConf()); } } } diff --git a/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java b/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java deleted file mode 100644 index dbee78e5ba..0000000000 --- a/tajo-core/src/main/java/org/apache/tajo/util/QueryContextUtil.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.util; - -import org.apache.tajo.conf.TajoConf; -import org.apache.tajo.engine.query.QueryContext; - -public class QueryContextUtil { - public static void updatePythonScriptPath(TajoConf systemConf, QueryContext queryContext) { - queryContext.setPythonScriptPath(systemConf.getVar(TajoConf.ConfVars.PYTHON_CODE_DIR)); - } -} diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java index d66474516d..17af71a5a4 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/TajoWorker.java @@ -237,7 +237,7 @@ public void serviceInit(Configuration conf) throws Exception { historyReader = new HistoryReader(workerContext.getWorkerName(), this.systemConf); - FunctionLoader.loadOptionalFunctions(systemConf, new HashMap()); + FunctionLoader.loadUserDefinedFunctions(systemConf, new HashMap()); diagnoseTajoWorker(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java index db0833fa89..a983f78426 100644 --- a/tajo-core/src/main/java/org/apache/tajo/worker/Task.java +++ b/tajo-core/src/main/java/org/apache/tajo/worker/Task.java @@ -46,7 +46,6 @@ import org.apache.tajo.ipc.TajoWorkerProtocol.*; import org.apache.tajo.ipc.TajoWorkerProtocol.EnforceProperty.EnforceType; import org.apache.tajo.master.cluster.WorkerConnectionInfo; -import org.apache.tajo.plan.function.python.PythonScriptEngine; import org.apache.tajo.plan.function.python.TajoScriptEngine; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.serder.LogicalNodeDeserializer; @@ -58,7 +57,6 @@ import org.apache.tajo.storage.*; import org.apache.tajo.storage.fragment.FileFragment; import org.apache.tajo.util.NetUtils; -import org.apache.tajo.util.QueryContextUtil; import java.io.File; import java.io.IOException; @@ -102,8 +100,6 @@ public class Task { private Schema finalSchema = null; private TupleComparator sortComp = null; - private PythonScriptEngine pythonEngine; - public Task(String taskRunnerId, Path baseDir, TaskAttemptId taskId, @@ -138,7 +134,6 @@ public Task(String taskRunnerId, } public void initPlan() throws IOException { - QueryContextUtil.updatePythonScriptPath(systemConf, queryContext); plan = LogicalNodeDeserializer.deserialize(queryContext, context.getEvalContext(), request.getPlan()); LogicalNode [] scanNode = PlannerUtil.findAllNodes(plan, NodeType.SCAN); if (scanNode != null) { @@ -196,7 +191,7 @@ public void initPlan() throws IOException { private void startScriptExecutors() throws IOException { for (TajoScriptEngine executor : context.getEvalContext().getAllScriptEngines()) { - executor.start(queryContext); + executor.start(systemConf); } } diff --git a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java index 9d6b1121a0..36ffd0c383 100644 --- a/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java +++ b/tajo-core/src/test/java/org/apache/tajo/engine/eval/ExprTestBase.java @@ -55,7 +55,6 @@ import org.apache.tajo.util.BytesUtils; import org.apache.tajo.util.CommonTestingUtil; import org.apache.tajo.util.KeyValueSet; -import org.apache.tajo.util.QueryContextUtil; import org.apache.tajo.util.datetime.DateTimeUtil; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -97,7 +96,7 @@ public static void setUp() throws Exception { cat.createTablespace(DEFAULT_TABLESPACE_NAME, "hdfs://localhost:1234/warehouse"); cat.createDatabase(DEFAULT_DATABASE_NAME, DEFAULT_TABLESPACE_NAME); Map map = FunctionLoader.load(); - map = FunctionLoader.loadOptionalFunctions(conf, map); + map = FunctionLoader.loadUserDefinedFunctions(conf, map); for (FunctionDesc funcDesc : map.values()) { cat.createFunction(funcDesc); } @@ -226,7 +225,6 @@ public void testEval(OverridableConf context, Schema schema, String tableName, S queryContext = LocalTajoTestingUtility.createDummyContext(conf); queryContext.putAll(context); } - QueryContextUtil.updatePythonScriptPath(conf, queryContext); String timezoneId = queryContext.get(SessionVars.TIMEZONE); TimeZone timeZone = TimeZone.getTimeZone(timezoneId); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java index 942c3b47cd..a8b09458fc 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/exprrewrite/rules/ConstantFolding.java @@ -94,7 +94,7 @@ public EvalNode visitFuncCall(LogicalPlanner.PlanContext context, FunctionEval e if (evalNode.getFuncDesc().getInvocation().hasPython()) { TajoScriptEngine executor = new PythonScriptEngine(evalNode.getFuncDesc()); try { - executor.start(context.getQueryContext()); + executor.start(context.getQueryContext().getConf()); EvalContext evalContext = new EvalContext(); evalContext.addScriptEngine(evalNode, executor); evalNode.bind(evalContext, null); diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java index d081665fa8..0da30f14b4 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/PythonScriptEngine.java @@ -21,8 +21,7 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.tajo.OverridableConf; -import org.apache.tajo.QueryVars; +import org.apache.hadoop.conf.Configuration; import org.apache.tajo.catalog.*; import org.apache.tajo.catalog.proto.CatalogProtos; import org.apache.tajo.common.TajoDataTypes; @@ -170,7 +169,7 @@ private static List getFunctions(InputStream is) throws IOException { private static final int CONTROLLER_LOG_FILE_PATH = 8; // Controller log file logs progress through the controller script not user code. private static final int OUT_SCHEMA = 9; // the schema of the output column - private OverridableConf queryContext; + private Configuration systemConf; private Process process; // Handle to the external execution of python functions @@ -207,8 +206,9 @@ public PythonScriptEngine(FunctionDesc functionDesc) { outSchema = new Schema(new Column[]{new Column("out", functionSignature.getReturnType())}); } - public void start(OverridableConf queryContext) throws IOException { - this.queryContext = queryContext; + @Override + public void start(Configuration systemConf) throws IOException { + this.systemConf = systemConf; startUdfController(); createInputHandlers(); setStreams(); @@ -217,6 +217,7 @@ public void start(OverridableConf queryContext) throws IOException { } } + @Override public void shutdown() { process.destroy(); FileUtil.cleanup(LOG, stdin); @@ -247,10 +248,10 @@ private String[] buildCommand() throws IOException { String[] command = new String[10]; // TODO: support controller logging - String standardOutputRootWriteLocation = ""; - if (queryContext.containsKey(QueryVars.PYTHON_CONTROLLER_LOG_DIR)) { + String standardOutputRootWriteLocation = systemConf.get(TajoConf.ConfVars.PYTHON_CONTROLLER_LOG_DIR.keyname(), + DEFAULT_LOG_DIR); + if (!standardOutputRootWriteLocation.equals(DEFAULT_LOG_DIR)) { LOG.warn("Currently, logging is not supported for the python controller."); - standardOutputRootWriteLocation = queryContext.get(QueryVars.PYTHON_CONTROLLER_LOG_DIR, DEFAULT_LOG_DIR); } String controllerLogFileName, outFileName, errOutFileName; @@ -269,10 +270,10 @@ private String[] buildCommand() throws IOException { command[UDF_FILE_NAME] = fileName; command[UDF_FILE_PATH] = lastSeparator <= 0 ? "." : filePath.substring(0, lastSeparator - 1); command[UDF_NAME] = funcName; - if (!queryContext.containsKey(QueryVars.PYTHON_SCRIPT_CODE_DIR)) { + String fileCachePath = systemConf.get(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname()); + if (fileCachePath == null) { throw new IOException(TajoConf.ConfVars.PYTHON_CODE_DIR.keyname() + " must be set."); } - String fileCachePath = queryContext.get(QueryVars.PYTHON_SCRIPT_CODE_DIR); command[PATH_TO_FILE_CACHE] = "'" + fileCachePath + "'"; command[STD_OUT_OUTPUT_PATH] = outFileName; command[STD_ERR_OUTPUT_PATH] = errOutFileName; diff --git a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java index 2ba9c2b4d2..726ec2faf7 100644 --- a/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java +++ b/tajo-plan/src/main/java/org/apache/tajo/plan/function/python/TajoScriptEngine.java @@ -18,7 +18,7 @@ package org.apache.tajo.plan.function.python; -import org.apache.tajo.OverridableConf; +import org.apache.hadoop.conf.Configuration; import org.apache.tajo.datum.Datum; import org.apache.tajo.storage.Tuple; @@ -62,10 +62,10 @@ protected static InputStream getScriptAsStream(URI scriptPath) { /** * Start TajoScriptEngine. * - * @param queryContext + * @param systemConf * @throws IOException */ - public abstract void start(OverridableConf queryContext) throws IOException; + public abstract void start(Configuration systemConf) throws IOException; /** * Shutdown TajoScriptEngine.