You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tajo.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2016/02/04 10:49:39 UTC

[jira] [Commented] (TAJO-1686) Allow Tajo to use Hive UDF

    [ https://issues.apache.org/jira/browse/TAJO-1686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15132050#comment-15132050 ] 

ASF GitHub Bot commented on TAJO-1686:
--------------------------------------

Github user jihoonson commented on a diff in the pull request:

    https://github.com/apache/tajo/pull/929#discussion_r51849575
  
    --- Diff: tajo-plan/src/main/java/org/apache/tajo/plan/function/HiveFunctionInvoke.java ---
    @@ -0,0 +1,95 @@
    +/**
    + * 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.hadoop.io.*;
    +import org.apache.tajo.catalog.FunctionDesc;
    +import org.apache.tajo.datum.Datum;
    +import org.apache.tajo.exception.TajoInternalError;
    +import org.apache.tajo.function.UDFInvocationDesc;
    +import org.apache.tajo.storage.Tuple;
    +import org.apache.tajo.util.WritableTypeConverter;
    +
    +import java.io.IOException;
    +import java.lang.reflect.Constructor;
    +import java.lang.reflect.InvocationTargetException;
    +import java.lang.reflect.Method;
    +import java.net.URL;
    +import java.net.URLClassLoader;
    +
    +public class HiveFunctionInvoke extends FunctionInvoke implements Cloneable {
    +  private Object instance = null;
    +  private Method evalMethod = null;
    +
    +  public HiveFunctionInvoke(FunctionDesc desc) {
    +    super(desc);
    +  }
    +
    +  @Override
    +  public void init(FunctionInvokeContext context) throws IOException {
    +    UDFInvocationDesc udfDesc = functionDesc.getInvocation().getUDF();
    +
    +    URL [] urls = new URL [] { new URL(udfDesc.getPath()) };
    +    URLClassLoader loader = new URLClassLoader(urls);
    +
    +    try {
    +      Class<?> udfclass = loader.loadClass(udfDesc.getName());
    +      evalMethod = getEvaluateMethod(udfclass);
    +    } catch (ClassNotFoundException e) {
    +      e.printStackTrace();
    +    }
    +  }
    +
    +  private Method getEvaluateMethod(Class<?> clazz) {
    +    Constructor constructor = clazz.getConstructors()[0];
    +
    +    try {
    +      instance = constructor.newInstance();
    +    } catch (InstantiationException|IllegalAccessException|InvocationTargetException e) {
    +      throw new TajoInternalError(e);
    +    }
    +
    +    for (Method m: clazz.getMethods()) {
    +      if (m.getName().equals("evaluate")) {
    +        return m;
    --- End diff --
    
    If there are several functions of the same name which accept different arguments, this looks to return a wrong function.


> Allow Tajo to use Hive UDF
> --------------------------
>
>                 Key: TAJO-1686
>                 URL: https://issues.apache.org/jira/browse/TAJO-1686
>             Project: Tajo
>          Issue Type: New Feature
>          Components: Function/UDF
>            Reporter: Jaehwa Jung
>            Assignee: Jongyoung Park
>
> Hive has been widely used in this area. Many users have maintained lots of big tables through Hive metastore using HiveQL and UDFs. Currently, Tajo provides own UDF and Hive users can implement their UDFs in Tajo. But if we can wrap Hive UDF in Tajo, it seems that they would be able to use Tajo easily for their analysis infrastructure.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)