You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2014/05/07 04:02:48 UTC

git commit: DRILL-219: Math Trigonometric functions

Repository: incubator-drill
Updated Branches:
  refs/heads/master 08923cb82 -> 85d52c744


DRILL-219: Math Trigonometric functions


Project: http://git-wip-us.apache.org/repos/asf/incubator-drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-drill/commit/85d52c74
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/85d52c74
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/85d52c74

Branch: refs/heads/master
Commit: 85d52c744753b54964d655af2e2a7e45f14c989c
Parents: 08923cb
Author: Yash Sharma <ya...@snapdeal.com>
Authored: Tue May 6 19:02:38 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Tue May 6 19:02:38 2014 -0700

----------------------------------------------------------------------
 .../src/main/codegen/data/MathFunc.tdd          | 124 ++++++++++++++++++-
 .../main/codegen/templates/MathFunctions.java   |  61 +++++++++
 .../exec/fn/impl/TestNewMathFunctions.java      | 119 ++++++++++++++++++
 .../functions/testTrigoMathFunctions.json       |  40 ++++++
 4 files changed, 342 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/85d52c74/exec/java-exec/src/main/codegen/data/MathFunc.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/MathFunc.tdd b/exec/java-exec/src/main/codegen/data/MathFunc.tdd
index 0ee94c2..228d207 100644
--- a/exec/java-exec/src/main/codegen/data/MathFunc.tdd
+++ b/exec/java-exec/src/main/codegen/data/MathFunc.tdd
@@ -136,5 +136,125 @@
      ] 
     }
     
-  ]  
-}
+  ],
+
+trigoMathFunctions : [
+ {className: "Sin", funcName: "sin", javaFunc : "java.lang.Math.sin", outputType: "Float8", types: [
+      {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "Cos", funcName: "cos", javaFunc : "java.lang.Math.cos", outputType: "Float8", types: [
+     {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "Tan", funcName: "tan", javaFunc : "java.lang.Math.tan", outputType: "Float8", types: [
+     {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "ASin", funcName: "asin", javaFunc : "java.lang.Math.asin", outputType: "Float8", types: [
+      {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "ACos", funcName: "acos", javaFunc : "java.lang.Math.acos", outputType: "Float8", types: [
+     {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "ATan", funcName: "atan", javaFunc : "java.lang.Math.atan", outputType: "Float8", types: [
+     {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "Sinh", funcName: "sinh", javaFunc : "java.lang.Math.sinh", outputType: "Float8", types: [
+      {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "Cosh", funcName: "cosh", javaFunc : "java.lang.Math.cosh", outputType: "Float8", types: [
+      {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+     ]
+ },
+ {className: "Tanh", funcName: "tanh", javaFunc : "java.lang.Math.tanh", outputType: "Float8", types: [
+      {input: "Int"},
+      {input: "BigInt"},
+      {input: "Float4"},
+      {input: "Float8"},
+      {input: "SmallInt"},
+      {input: "TinyInt"},
+      {input: "UInt1"},
+      {input: "UInt2"},
+      {input: "UInt4"},
+      {input: "UInt8"}
+    ]
+ }
+]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/85d52c74/exec/java-exec/src/main/codegen/templates/MathFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/MathFunctions.java b/exec/java-exec/src/main/codegen/templates/MathFunctions.java
index 66373f5..ac7cedb 100644
--- a/exec/java-exec/src/main/codegen/templates/MathFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/MathFunctions.java
@@ -97,3 +97,64 @@ public class GMathFunctions{
   </#list>
   </#list>
 }
+
+//////////////////////////////////////////////////////////////////////////////////////////////////
+//End of GMath Functions
+//////////////////////////////////////////////////////////////////////////////////////////////////
+
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/TrigoMathFunctions.java" />
+<#include "/@includes/license.ftl" />
+
+//////////////////////////////////////////////////////////////////////////////////////////////////
+//Functions for Trigo Math Functions
+//////////////////////////////////////////////////////////////////////////////////////////////////
+
+
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.FunctionScope;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.fn.impl.StringFunctions;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+
+/*
+ * This class is automatically generated from MathFunc.tdd using FreeMarker.
+ */
+
+@SuppressWarnings("unused")
+
+public class TrigoMathFunctions{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TrigoMathFunctions.class);
+
+  private TrigoMathFunctions(){}
+
+  <#list mathFunc.trigoMathFunctions as func>
+
+  <#list func.types as type>
+
+  @FunctionTemplate(name = "${func.funcName}", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class ${func.className}${type.input} implements DrillSimpleFunc {
+
+    @Param ${type.input}Holder in;
+    @Output ${func.outputType}Holder out;
+
+    public void setup(RecordBatch b) {
+    }
+
+    public void eval() {
+      out.value = ${func.javaFunc}(in.value);
+    }
+  }
+ </#list>
+ </#list>
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/85d52c74/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java
new file mode 100644
index 0000000..3a67d31
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestNewMathFunctions.java
@@ -0,0 +1,119 @@
+/**
+ * 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.drill.exec.fn.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import mockit.Injectable;
+import mockit.NonStrictExpectations;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.FragmentRoot;
+import org.apache.drill.exec.physical.impl.ImplCreator;
+import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
+import org.apache.drill.exec.physical.impl.SimpleRootExec;
+import org.apache.drill.exec.physical.impl.TestStringFunctions;
+import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.proto.CoordinationProtos;
+import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.rpc.user.UserServer;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.junit.Test;
+
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.base.Charsets;
+import com.google.common.io.Resources;
+
+public class TestNewMathFunctions {
+
+
+	static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestNewMathFunctions.class);
+
+	  DrillConfig c = DrillConfig.create();
+	  PhysicalPlanReader reader;
+	  FunctionImplementationRegistry registry;
+	  FragmentContext context;
+
+	  public Object[] getRunResult(SimpleRootExec exec) {
+	    int size = 0;
+	    for (ValueVector v : exec) {
+	      size++;
+	    }
+
+	    Object[] res = new Object [size];
+	    int i = 0;
+	    for (ValueVector v : exec) {
+	      if  (v instanceof VarCharVector) {
+	        res[i++] = new String( ((VarCharVector) v).getAccessor().get(0));
+	      } else
+	        res[i++] =  v.getAccessor().getObject(0);
+	    }
+	    return res;
+	 }
+
+	  public void runTest(@Injectable final DrillbitContext bitContext,
+	                      @Injectable UserServer.UserClientConnection connection, Object[] expectedResults, String planPath) throws Throwable {
+
+	    new NonStrictExpectations(){{
+	      bitContext.getMetrics(); result = new MetricRegistry();
+	      bitContext.getAllocator(); result = new TopLevelAllocator();
+	      bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
+	    }};
+
+	    String planString = Resources.toString(Resources.getResource(planPath), Charsets.UTF_8);
+	    if(reader == null) reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
+	    if(registry == null) registry = new FunctionImplementationRegistry(c);
+	    if(context == null) context =  new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry); //new FragmentContext(bitContext, ExecProtos.FragmentHandle.getDefaultInstance(), connection, registry);
+	    PhysicalPlan plan = reader.readPhysicalPlan(planString);
+	    SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
+
+	    while(exec.next()){
+	      Object [] res = getRunResult(exec);
+	      assertEquals("return count does not match", res.length, expectedResults.length);
+
+	      System.out.println("-----------------------------------------------");
+	      System.out.println("ACTUAL_RESULTS\t\tEXPECTED_RESULTS");
+	      System.out.println("-----------------------------------------------");
+	      for (int i = 0; i<res.length; i++) {
+	      System.out.println(res[i] + "\t" + expectedResults[i]);
+	        assertEquals(String.format("column %s does not match", i),  res[i], expectedResults[i]);
+	      }
+	      System.out.println("-----------------------------------------------");
+	    }
+
+	    if(context.getFailureCause() != null){
+	      throw context.getFailureCause();
+	    }
+
+	    assertTrue(!context.isFailed());
+	  }
+
+      @Test
+	  public void testTrigoMathFunc(@Injectable final DrillbitContext bitContext,
+	                           @Injectable UserServer.UserClientConnection connection) throws Throwable{
+	    Object [] expected = new Object[] {Math.sin(45), Math.cos(45), Math.tan(45),Math.asin(45), Math.acos(45), Math.atan(45),Math.sinh(45), Math.cosh(45), Math.tanh(45)};
+	    runTest(bitContext, connection, expected, "functions/testTrigoMathFunctions.json");
+	  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/85d52c74/exec/java-exec/src/test/resources/functions/testTrigoMathFunctions.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/testTrigoMathFunctions.json b/exec/java-exec/src/test/resources/functions/testTrigoMathFunctions.json
new file mode 100644
index 0000000..c5a3f5c
--- /dev/null
+++ b/exec/java-exec/src/test/resources/functions/testTrigoMathFunctions.json
@@ -0,0 +1,40 @@
+{
+  head : {
+    version : 1,
+    generator : {
+      type : "optiq",
+      info : "na"
+    },
+    type : "APACHE_DRILL_PHYSICAL"
+  },
+         graph:[
+         {
+             @id:1,
+             pop:"mock-sub-scan",
+             url: "http://apache.org",
+             entries:[
+                 {records: 1, types: [
+                   {name: "blue", type: "BIGINT", mode: "REQUIRED"}
+                 ]}
+             ]
+         }, {
+    pop : "project",
+    @id : 2,
+    exprs : [
+    { ref : "SIN_VAL", expr : " sin(45) "},
+    { ref : "COS_VAL", expr : " cos(45) "},
+    { ref : "TAN_VAL", expr : " tan(45) " },
+    { ref : "ASIN_VAL", expr : " asin(45) "},
+    { ref : "ACOS_VAL", expr : " acos(45) "},
+    { ref : "ATAN_VAL", expr : " atan(45) " },
+    { ref : "SINH_VAL", expr : " sinh(45) "},
+    { ref : "COSH_VAL", expr : " cosh(45) "},
+    { ref : "TANH_VAL", expr : " tanh(45) " }
+    ],
+    child : 1
+  }, {
+    pop : "screen",
+    @id : 3,
+    child : 2
+  } ]
+}
\ No newline at end of file