[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-13 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105650856
  
--- Diff: 
flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/table/hive/functions/HiveSimpleUDF.scala
 ---
@@ -0,0 +1,87 @@
+/*
+ * 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.flink.table.hive.functions
+
+import java.lang.reflect.Method
+import java.math.BigDecimal
+import java.util
+
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, UDF}
+import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
+import org.apache.hadoop.hive.serde2.typeinfo.{PrimitiveTypeInfo, 
TypeInfo, TypeInfoFactory}
+
+import scala.annotation.varargs
+
+/**
+  * A Hive UDF Wrapper which behaves as a Flink-table ScalarFunction.
+  *
+  * This class has to have a method with @varargs annotation. For scala 
will compile
+  *  eval(args: Any*)  to eval(args: Seq).
+  * This will cause an exception in Janino compiler.
+  */
+class HiveSimpleUDF(className: String) extends ScalarFunction {
+
+  @transient
+  private lazy val functionWrapper = HiveFunctionWrapper(className)
+
+  @transient
+  private lazy val function = functionWrapper.createFunction[UDF]()
+
+  @transient
+  private var typeInfos: util.List[TypeInfo] = _
+
+  @transient
+  private var objectInspectors: Array[ObjectInspector] = _
+
+  @transient
+  private var conversionHelper: ConversionHelper = _
+
+  @transient
+  private var method: Method = _
+
+  @varargs
+  def eval(args: AnyRef*) : Any = {
+if (null == typeInfos) {
--- End diff --

The type checking logic needs to be done before runtime. The user should 
get an exception before sending the job to the cluster. ScalarFunctions can 
have an open() method now, we can move runtime logic there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-13 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105646577
  
--- Diff: 
flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/table/hive/functions/HiveSimpleUDF.scala
 ---
@@ -0,0 +1,87 @@
+/*
+ * 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.flink.table.hive.functions
+
+import java.lang.reflect.Method
+import java.math.BigDecimal
+import java.util
+
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, UDF}
+import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
+import org.apache.hadoop.hive.serde2.typeinfo.{PrimitiveTypeInfo, 
TypeInfo, TypeInfoFactory}
+
+import scala.annotation.varargs
+
+/**
+  * A Hive UDF Wrapper which behaves as a Flink-table ScalarFunction.
+  *
+  * This class has to have a method with @varargs annotation. For scala 
will compile
+  *  eval(args: Any*)  to eval(args: Seq).
+  * This will cause an exception in Janino compiler.
+  */
+class HiveSimpleUDF(className: String) extends ScalarFunction {
--- End diff --

We should also allow passing UDF classes directly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-13 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105638573
  
--- Diff: flink-connectors/flink-hcatalog/pom.xml ---
@@ -44,14 +44,20 @@ under the License.
 

org.apache.flink
+   flink-table_2.10
+   ${project.version}
--- End diff --

The pom.xml should be similar to e.g. `flink-connector-kafka-base`:
```

org.apache.flink
flink-table_2.10
${project.version}
provided

true

```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-13 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105646877
  
--- Diff: 
flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/table/hive/functions/HiveSimpleUDF.scala
 ---
@@ -0,0 +1,87 @@
+/*
+ * 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.flink.table.hive.functions
+
+import java.lang.reflect.Method
+import java.math.BigDecimal
+import java.util
+
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, UDF}
+import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
+import org.apache.hadoop.hive.serde2.typeinfo.{PrimitiveTypeInfo, 
TypeInfo, TypeInfoFactory}
+
+import scala.annotation.varargs
+
+/**
+  * A Hive UDF Wrapper which behaves as a Flink-table ScalarFunction.
--- End diff --

The JavaDoc should be extended a little bit. What types are supported? 
Restrictions etc. This is a user-facing class, so no comments about Janino etc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-13 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105649084
  
--- Diff: 
flink-connectors/flink-hcatalog/src/test/scala/org/apache/flink/table/hive/functions/utils/ExpressionTestBase.scala
 ---
@@ -0,0 +1,270 @@
+/*
+ * 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.flink.table.hive.functions.utils
+
+import java.util
+import java.util.concurrent.Future
+
+import org.apache.calcite.plan.hep.{HepMatchOrder, HepPlanner, 
HepProgramBuilder}
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import org.apache.calcite.sql2rel.RelDecorrelator
+import org.apache.calcite.tools.{Programs, RelBuilder}
+import org.apache.flink.api.common.TaskInfo
+import org.apache.flink.api.common.accumulators.Accumulator
+import org.apache.flink.api.common.functions._
+import org.apache.flink.api.common.functions.util.RuntimeUDFContext
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.java.{DataSet => JDataSet}
+import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.core.fs.Path
+import org.apache.flink.table.api.{BatchTableEnvironment, TableConfig, 
TableEnvironment}
+import org.apache.flink.table.calcite.FlinkPlannerImpl
+import org.apache.flink.table.codegen.{CodeGenerator, Compiler, 
GeneratedFunction}
+import org.apache.flink.table.expressions.{Expression, ExpressionParser}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.plan.nodes.dataset.{DataSetCalc, 
DataSetConvention}
+import org.apache.flink.table.plan.rules.FlinkRuleSets
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit.{After, Before}
+import org.mockito.Mockito._
+
+import scala.collection.mutable
+
+/**
+  * Base test class for expression tests.
+  */
+abstract class ExpressionTestBase {
--- End diff --

You can add a Table API Maven dependency for the `test` scope to access 
this class instead of copying.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-13 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105638967
  
--- Diff: flink-connectors/flink-hcatalog/pom.xml ---
@@ -44,14 +44,20 @@ under the License.
 

org.apache.flink
+   flink-table_2.10
+   ${project.version}
+   
+
+   
+   org.apache.flink
flink-hadoop-compatibility_2.10
${project.version}

 

org.apache.hive.hcatalog
-   hcatalog-core
-   0.12.0
+   hive-hcatalog-core
--- End diff --

@fhueske @rmetzger Is it ok to upgrade the hcatalog version? It is pretty 
old anyway.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-13 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105648552
  
--- Diff: 
flink-connectors/flink-hcatalog/src/main/scala/org/apache/flink/table/hive/functions/HiveSimpleUDF.scala
 ---
@@ -0,0 +1,87 @@
+/*
+ * 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.flink.table.hive.functions
+
+import java.lang.reflect.Method
+import java.math.BigDecimal
+import java.util
+
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, UDF}
+import 
org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector
+import 
org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory
+import org.apache.hadoop.hive.serde2.typeinfo.{PrimitiveTypeInfo, 
TypeInfo, TypeInfoFactory}
+
+import scala.annotation.varargs
+
+/**
+  * A Hive UDF Wrapper which behaves as a Flink-table ScalarFunction.
+  *
+  * This class has to have a method with @varargs annotation. For scala 
will compile
+  *  eval(args: Any*)  to eval(args: Seq).
+  * This will cause an exception in Janino compiler.
+  */
+class HiveSimpleUDF(className: String) extends ScalarFunction {
+
+  @transient
+  private lazy val functionWrapper = HiveFunctionWrapper(className)
+
+  @transient
+  private lazy val function = functionWrapper.createFunction[UDF]()
+
+  @transient
+  private var typeInfos: util.List[TypeInfo] = _
+
+  @transient
+  private var objectInspectors: Array[ObjectInspector] = _
+
+  @transient
+  private var conversionHelper: ConversionHelper = _
+
+  @transient
+  private var method: Method = _
+
+  @varargs
+  def eval(args: AnyRef*) : Any = {
+if (null == typeInfos) {
+  typeInfos = new util.ArrayList[TypeInfo]()
+  args.foreach(arg => {
+  
typeInfos.add(TypeInfoFactory.getPrimitiveTypeInfoFromJavaPrimitive(arg.getClass))
+  })
+  method = function.getResolver.getEvalMethod(typeInfos)
+
+  objectInspectors = new Array[ObjectInspector](typeInfos.size())
+  args.zipWithIndex.foreach { case (_, i) =>
+objectInspectors(i) = 
PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector(
+  typeInfos.get(i).asInstanceOf[PrimitiveTypeInfo])
+  }
+  conversionHelper = new ConversionHelper(method, objectInspectors)
+}
+
+val mappedArgs = args.map {
--- End diff --

Maybe it is better to throw an exception because of an unsupported type 
BigDecimal type, such that the user can manually cast the BigDecimal to double. 
Otherwise the user is not aware of losing precision.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-08 Thread clarkyzl
Github user clarkyzl commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r105095839
  
--- Diff: flink-libraries/flink-table/pom.xml ---
@@ -92,6 +92,11 @@ under the License.


 
+   
+   org.apache.hive
--- End diff --

When I rebased the patch onto #3389 , the issue disappears.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-03 Thread clarkyzl
Github user clarkyzl commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r104109600
  
--- Diff: flink-libraries/flink-table/pom.xml ---
@@ -92,6 +92,11 @@ under the License.


 
+   
+   org.apache.hive
--- End diff --

Hi @twalthr .After moved the code to flink-hcatalog, I found the `mvn clean 
verify` will fail the test. The reason is that the code depends on the 
FLINK-5881. However, the flink-hcatalog depends on the `1.3-SNAPSHOT` of the 
maven repository. The patch of FLINK-5881 wasn't available there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-03 Thread clarkyzl
Github user clarkyzl commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r104106388
  
--- Diff: flink-libraries/flink-table/pom.xml ---
@@ -92,6 +92,11 @@ under the License.


 
+   
+   org.apache.hive
--- End diff --

Hi @twalthr . Thank you for your suggestions. I've moved all the code to 
hcatalog in commit 97968fb86af105f0fc8b68e8083998dd0725deef.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-02 Thread twalthr
Github user twalthr commented on a diff in the pull request:

https://github.com/apache/flink/pull/3456#discussion_r103942004
  
--- Diff: flink-libraries/flink-table/pom.xml ---
@@ -92,6 +92,11 @@ under the License.


 
+   
+   org.apache.hive
--- End diff --

I haven't had a look at the entire code yet, but the `flink-table` should 
not have a dependency to Hive. We should move the Hive dependent code to 
`flink-hcatalog`. Users that want to use any Hive functionality can then add 
this module to their dependencies. Similar to `KafkaTableSource`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] flink pull request #3456: [FLINK-5832] [table] Support for simple hive UDF

2017-03-02 Thread clarkyzl
GitHub user clarkyzl opened a pull request:

https://github.com/apache/flink/pull/3456

[FLINK-5832] [table] Support for simple hive UDF

Type: New Feature
Priority: Major
Components: table, udf, hive
Problem definition: Make Flink call Hive User-Defined Functions, support 
for simple hive UDFs.

Design:
1. This patch is based on FLINK-5881, we need variable arguments to call 
hive udfs.
2. Added a HiveFunctionWrapper to create Hive simple UDFs.
3. Added a ScalarFunction called HiveSimpleUDF to call Hive Simple UDFs. 
Use primitive java object inspectors for the simple UDFs.
4. A few modification of type comparation, make type of Object equal any 
types.
5. Added a new method in TableEnvironment to rigister hive UDFs.

Impact Analysis:
A new feature, had few impacts on exsting features, except for the 
comparation of types.

Test:
`mvn clean verify` is done on local.


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/clarkyzl/flink udf

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/flink/pull/3456.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #3456


commit 60b68fdd66f8021f6f090e7372987d43362d5ef3
Author: Zhuoluo Yang 
Date:   2017-02-22T10:53:34Z

[FLINK-5881] [table] ScalarFunction(UDF) should support variable types and 
variable arguments

commit fe81a7d87d2620d19e5dd0fa569f139569b6c2aa
Author: Zhuoluo Yang 
Date:   2017-02-23T11:29:37Z

[FLINK-5881] [table] Modification as reviews

commit 21f8a4ec40a12828e59ff959b56d92c2c2629afd
Author: Zhuoluo Yang 
Date:   2017-02-20T06:04:12Z

[FLINK-5832] [table] Support for simple hive UDF

commit d734867874169ffd23f084fbdd7f8605208f2b37
Author: Zhuoluo Yang 
Date:   2017-02-27T05:24:33Z

[FLINK-5832] [table] Delete test code

commit 3b345f7d19054b7cbb961498d3f817fcaae128b5
Author: Zhuoluo Yang 
Date:   2017-02-28T07:54:39Z

[FLINK-5832] [table] Use scala to implement HiveSimpleUDF

commit 8838965d0add8ae1e47d216909e20ccd3d7fcd17
Author: Zhuoluo Yang 
Date:   2017-02-28T07:56:36Z

[FLINK-5832] [table] Remove unlicensed files

commit 260bf3b302e5753845ed30fe89b1516121bc3562
Author: Zhuoluo Yang 
Date:   2017-02-28T09:22:01Z

[FLINK-5832] [table] match the signature, varargs

commit fcda8546509cb755e101ef21a12b905f97cadf72
Author: Zhuoluo Yang 
Date:   2017-03-01T07:30:15Z

[FLINK-5881] [table] Modification as Jark's two comments

commit 1eca2a511de8eeb734f029887f2e0831f9092a7b
Author: Zhuoluo Yang 
Date:   2017-03-01T08:13:40Z

Merge branch 'flink-5881' into udf

commit 13b2039fcc8785b3b27a3d1537daec1c69aec944
Author: Zhuoluo Yang 
Date:   2017-03-01T10:40:58Z

[FLINK-5832] [table] still needs some ois

commit 80123fe9f82c989f74ec1d5194bc1609cfd207aa
Author: Zhuoluo Yang 
Date:   2017-03-02T03:28:38Z

[FLINK-5832] [table] Use FunctionRegistry to call HiveSimpleUDFs

commit b1cb548aa9f89f1ceef896bd8583a65054190795
Author: Zhuoluo Yang 
Date:   2017-03-02T08:00:31Z

 [FLINK-5832] [table] Use PrimitiveObjectInspectors and add some unit tests




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---