You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2017/03/02 20:59:48 UTC

[21/27] drill git commit: DRILL-4963: Fix issues with dynamically loaded overloaded functions

http://git-wip-us.apache.org/repos/asf/drill/blob/dcbcb94f/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java b/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
index 10a03b7..25c01b8 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDynamicUDFSupport.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
@@ -134,7 +134,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
   @Test
   public void testAbsentSourceInStaging() throws Exception {
     Path staging = getDrillbitContext().getRemoteFunctionRegistry().getStagingArea();
-    copyJar(getDrillbitContext().getRemoteFunctionRegistry().getFs(), new Path(jars.toURI()), staging, default_binary_name);
+    copyJar(getDrillbitContext().getRemoteFunctionRegistry().getFs(), new Path(jars.toURI()),
+        staging, default_binary_name);
 
     String summary = String.format("File %s does not exist", new Path(staging, default_source_name).toUri().getPath());
 
@@ -157,7 +158,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
         .sqlQuery("create function using jar '%s'", jarWithNoMarkerFile)
         .unOrdered()
         .baselineColumns("ok", "summary")
-        .baselineValues(false, String.format(summary, CommonConstants.DRILL_JAR_MARKER_FILE_RESOURCE_PATHNAME, jarWithNoMarkerFile))
+        .baselineValues(false, String.format(summary,
+            CommonConstants.DRILL_JAR_MARKER_FILE_RESOURCE_PATHNAME, jarWithNoMarkerFile))
         .go();
   }
 
@@ -201,7 +203,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     assertTrue("Source should be present in registry area",
         fs.exists(new Path(remoteFunctionRegistry.getRegistryArea(), default_source_name)));
 
-    Registry registry = remoteFunctionRegistry.getRegistry();
+    Registry registry = remoteFunctionRegistry.getRegistry(new DataChangeVersion());
     assertEquals("Registry should contain one jar", registry.getJarList().size(), 1);
     assertEquals(registry.getJar(0).getName(), default_binary_name);
   }
@@ -304,7 +306,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     assertTrue("Source should be present in registry area",
             fs.exists(new Path(remoteFunctionRegistry.getRegistryArea(), default_source_name)));
 
-    Registry registry = remoteFunctionRegistry.getRegistry();
+    Registry registry = remoteFunctionRegistry.getRegistry(new DataChangeVersion());
     assertEquals("Registry should contain one jar", registry.getJarList().size(), 1);
     assertEquals(registry.getJar(0).getName(), default_binary_name);
   }
@@ -337,7 +339,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     FileSystem fs = remoteFunctionRegistry.getFs();
 
     assertFalse("Registry area should be empty", fs.listFiles(remoteFunctionRegistry.getRegistryArea(), false).hasNext());
-    assertEquals("Registry should be empty", remoteFunctionRegistry.getRegistry().getJarList().size(), 0);
+    assertEquals("Registry should be empty",
+        remoteFunctionRegistry.getRegistry(new DataChangeVersion()).getJarList().size(), 0);
   }
 
   @Test
@@ -367,10 +370,13 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     assertTrue("Source should be present in staging area",
             fs.exists(new Path(remoteFunctionRegistry.getStagingArea(), default_source_name)));
 
-    assertFalse("Registry area should be empty", fs.listFiles(remoteFunctionRegistry.getRegistryArea(), false).hasNext());
-    assertFalse("Temporary area should be empty", fs.listFiles(remoteFunctionRegistry.getTmpArea(), false).hasNext());
+    assertFalse("Registry area should be empty",
+        fs.listFiles(remoteFunctionRegistry.getRegistryArea(), false).hasNext());
+    assertFalse("Temporary area should be empty",
+        fs.listFiles(remoteFunctionRegistry.getTmpArea(), false).hasNext());
 
-    assertEquals("Registry should be empty", remoteFunctionRegistry.getRegistry().getJarList().size(), 0);
+    assertEquals("Registry should be empty",
+        remoteFunctionRegistry.getRegistry(new DataChangeVersion()).getJarList().size(), 0);
   }
 
   @Test
@@ -402,7 +408,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     assertTrue("Source should be present in registry area",
             fs.exists(new Path(remoteFunctionRegistry.getRegistryArea(), default_source_name)));
 
-    Registry registry = remoteFunctionRegistry.getRegistry();
+    Registry registry = remoteFunctionRegistry.getRegistry(new DataChangeVersion());
     assertEquals("Registry should contain one jar", registry.getJarList().size(), 1);
     assertEquals(registry.getJar(0).getName(), default_binary_name);
   }
@@ -424,7 +430,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
         .baselineValues("a")
         .go();
 
-    Path localUdfDirPath = Deencapsulation.getField(getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir");
+    Path localUdfDirPath = Deencapsulation.getField(
+        getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir");
     File localUdfDir = new File(localUdfDirPath.toUri().getPath());
 
     assertTrue("Binary should exist in local udf directory", new File(localUdfDir, default_binary_name).exists());
@@ -455,6 +462,33 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     }
   }
 
+  @Test
+  public void testOverloadedFunctionPlanningStage() throws Exception {
+    String jarName = "DrillUDF-overloading-1.0.jar";
+    copyJarsToStagingArea(jarName, JarUtil.getSourceName(jarName));
+    test("create function using jar '%s'", jarName);
+
+    testBuilder()
+        .sqlQuery("select abs('A', 'A') as res from (values(1))")
+        .unOrdered()
+        .baselineColumns("res")
+        .baselineValues("ABS was overloaded. Input: A, A")
+        .go();
+  }
+
+  @Test
+  public void testOverloadedFunctionExecutionStage() throws Exception {
+    String jarName = "DrillUDF-overloading-1.0.jar";
+    copyJarsToStagingArea(jarName, JarUtil.getSourceName(jarName));
+    test("create function using jar '%s'", jarName);
+
+    testBuilder()
+        .sqlQuery("select log('A') as res from (values(1))")
+        .unOrdered()
+        .baselineColumns("res")
+        .baselineValues("LOG was overloaded. Input: A")
+        .go();
+  }
 
   @Test
   public void testDropFunction() throws Exception {
@@ -462,7 +496,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     test("create function using jar '%s'", default_binary_name);
     test("select custom_lower('A') from (values(1))");
 
-    Path localUdfDirPath = Deencapsulation.getField(getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir");
+    Path localUdfDirPath = Deencapsulation.getField(
+        getDrillbitContext().getFunctionImplementationRegistry(), "localUdfDir");
     File localUdfDir = new File(localUdfDirPath.toUri().getPath());
 
     assertTrue("Binary should exist in local udf directory", new File(localUdfDir, default_binary_name).exists());
@@ -485,7 +520,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     }
 
     RemoteFunctionRegistry remoteFunctionRegistry = getDrillbitContext().getRemoteFunctionRegistry();
-    assertEquals("Remote registry should be empty", remoteFunctionRegistry.getRegistry().getJarList().size(), 0);
+    assertEquals("Remote registry should be empty",
+        remoteFunctionRegistry.getRegistry(new DataChangeVersion()).getJarList().size(), 0);
 
     FileSystem fs = remoteFunctionRegistry.getFs();
     assertFalse("Binary should not be present in registry area",
@@ -561,8 +597,10 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
         .baselineValues(false, errorMessage)
         .go();
 
-    assertFalse("Registry area should be empty", fs.listFiles(remoteFunctionRegistry.getRegistryArea(), false).hasNext());
-    assertFalse("Temporary area should be empty", fs.listFiles(remoteFunctionRegistry.getTmpArea(), false).hasNext());
+    assertFalse("Registry area should be empty",
+        fs.listFiles(remoteFunctionRegistry.getRegistryArea(), false).hasNext());
+    assertFalse("Temporary area should be empty",
+        fs.listFiles(remoteFunctionRegistry.getTmpArea(), false).hasNext());
 
     assertTrue("Binary should be present in staging area",
         fs.exists(new Path(remoteFunctionRegistry.getStagingArea(), default_binary_name)));
@@ -684,7 +722,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
 
     DataChangeVersion version = new DataChangeVersion();
     Registry registry = remoteFunctionRegistry.getRegistry(version);
-    assertEquals("Remote registry version should match", 2, version.getVersion());
+    assertEquals("Remote registry version should match", 1, version.getVersion());
     List<Jar> jarList = registry.getJarList();
     assertEquals("Only one jar should be registered", 1, jarList.size());
     assertEquals("Jar name should match", jarName1, jarList.get(0).getName());
@@ -748,7 +786,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
 
     DataChangeVersion version = new DataChangeVersion();
     Registry registry = remoteFunctionRegistry.getRegistry(version);
-    assertEquals("Remote registry version should match", 3, version.getVersion());
+    assertEquals("Remote registry version should match", 2, version.getVersion());
 
     List<Jar> actualJars = registry.getJarList();
     List<String> expectedJars = Lists.newArrayList(jarName1, jarName2);
@@ -777,7 +815,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
       public Boolean answer(InvocationOnMock invocation) throws Throwable {
         latch1.await();
         boolean result = (boolean) invocation.callRealMethod();
-        assertTrue("loadRemoteFunctions() should return true", result);
+        assertTrue("syncWithRemoteRegistry() should return true", result);
         latch2.countDown();
         return true;
       }
@@ -788,11 +826,11 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
             latch1.countDown();
             latch2.await();
             boolean result = (boolean) invocation.callRealMethod();
-            assertTrue("loadRemoteFunctions() should return true", result);
+            assertTrue("syncWithRemoteRegistry() should return true", result);
             return true;
           }
         })
-        .when(functionImplementationRegistry).loadRemoteFunctions(anyLong());
+        .when(functionImplementationRegistry).syncWithRemoteRegistry(anyLong());
 
     SimpleQueryRunner simpleQueryRunner = new SimpleQueryRunner(query);
     Thread thread1 = new Thread(simpleQueryRunner);
@@ -804,9 +842,10 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
     thread1.join();
     thread2.join();
 
-    verify(functionImplementationRegistry, times(2)).loadRemoteFunctions(anyLong());
-    LocalFunctionRegistry localFunctionRegistry = Deencapsulation.getField(functionImplementationRegistry, "localFunctionRegistry");
-    assertEquals("Local functionRegistry version should match", 2L, localFunctionRegistry.getVersion());
+    verify(functionImplementationRegistry, times(2)).syncWithRemoteRegistry(anyLong());
+    LocalFunctionRegistry localFunctionRegistry = Deencapsulation.getField(
+        functionImplementationRegistry, "localFunctionRegistry");
+    assertEquals("Sync function registry version should match", 1L, localFunctionRegistry.getVersion());
   }
 
   @Test
@@ -819,7 +858,7 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
       @Override
       public Boolean answer(InvocationOnMock invocation) throws Throwable {
         boolean result = (boolean) invocation.callRealMethod();
-        assertTrue("loadRemoteFunctions() should return true", result);
+        assertTrue("syncWithRemoteRegistry() should return true", result);
         return true;
       }
     })
@@ -827,11 +866,11 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
           @Override
           public Boolean answer(InvocationOnMock invocation) throws Throwable {
             boolean result = (boolean) invocation.callRealMethod();
-            assertFalse("loadRemoteFunctions() should return false", result);
+            assertFalse("syncWithRemoteRegistry() should return false", result);
             return false;
           }
         })
-        .when(functionImplementationRegistry).loadRemoteFunctions(anyLong());
+        .when(functionImplementationRegistry).syncWithRemoteRegistry(anyLong());
 
     test("select custom_lower('A') from (values(1))");
 
@@ -841,9 +880,10 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
       assertThat(e.getMessage(), containsString("No match found for function signature unknown_lower(<CHARACTER>)"));
     }
 
-    verify(functionImplementationRegistry, times(2)).loadRemoteFunctions(anyLong());
-    LocalFunctionRegistry localFunctionRegistry = Deencapsulation.getField(functionImplementationRegistry, "localFunctionRegistry");
-    assertEquals("Local functionRegistry version should match", 2L, localFunctionRegistry.getVersion());
+    verify(functionImplementationRegistry, times(2)).syncWithRemoteRegistry(anyLong());
+    LocalFunctionRegistry localFunctionRegistry = Deencapsulation.getField(
+        functionImplementationRegistry, "localFunctionRegistry");
+    assertEquals("Sync function registry version should match", 1L, localFunctionRegistry.getVersion());
   }
 
   private void copyDefaultJarsToStagingArea() throws IOException {
@@ -866,7 +906,8 @@ public class TestDynamicUDFSupport extends BaseTestQuery {
   }
 
   private RemoteFunctionRegistry spyRemoteFunctionRegistry() {
-    FunctionImplementationRegistry functionImplementationRegistry = getDrillbitContext().getFunctionImplementationRegistry();
+    FunctionImplementationRegistry functionImplementationRegistry =
+        getDrillbitContext().getFunctionImplementationRegistry();
     RemoteFunctionRegistry remoteFunctionRegistry = functionImplementationRegistry.getRemoteFunctionRegistry();
     RemoteFunctionRegistry spy = spy(remoteFunctionRegistry);
     Deencapsulation.setField(functionImplementationRegistry, "remoteFunctionRegistry", spy);

http://git-wip-us.apache.org/repos/asf/drill/blob/dcbcb94f/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java b/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java
index ab886c4..88f1fcb 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/coord/zk/TestZookeeperClient.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
@@ -22,19 +22,13 @@ import java.util.List;
 import java.util.Map;
 
 import com.google.common.collect.Lists;
-import org.apache.curator.CuratorZookeeperClient;
 import org.apache.curator.RetryPolicy;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.CuratorFrameworkFactory;
-import org.apache.curator.framework.api.ACLBackgroundPathAndBytesable;
-import org.apache.curator.framework.api.CreateBuilder;
-import org.apache.curator.framework.api.DeleteBuilder;
-import org.apache.curator.framework.api.SetDataBuilder;
 import org.apache.curator.framework.recipes.cache.ChildData;
 import org.apache.curator.framework.recipes.cache.PathChildrenCache;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.test.TestingServer;
-import org.apache.curator.utils.EnsurePath;
 import org.apache.drill.common.collections.ImmutableEntry;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.exception.VersionMismatchException;
@@ -47,7 +41,9 @@ import org.junit.Test;
 import org.mockito.Mockito;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 public class TestZookeeperClient {
   private final static String root = "/test";
@@ -132,6 +128,26 @@ public class TestZookeeperClient {
   }
 
   @Test
+  public void testHasPathTrueWithVersion() {
+    client.put(path, data);
+    DataChangeVersion version0 = new DataChangeVersion();
+    assertTrue(client.hasPath(path, true, version0));
+    assertEquals("Versions should match", 0, version0.getVersion());
+    client.put(path, data);
+    DataChangeVersion version1 = new DataChangeVersion();
+    assertTrue(client.hasPath(path, true, version1));
+    assertEquals("Versions should match", 1, version1.getVersion());
+  }
+
+  @Test
+  public void testHasPathFalseWithVersion() {
+    DataChangeVersion version0 = new DataChangeVersion();
+    version0.setVersion(-1);
+    assertFalse(client.hasPath("unknown_path", true, version0));
+    assertEquals("Versions should not have changed", -1, version0.getVersion());
+  }
+
+  @Test
   public void testPutAndGetWorks() {
     client.put(path, data);
     final byte[] actual = client.get(path, true);

http://git-wip-us.apache.org/repos/asf/drill/blob/dcbcb94f/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolderTest.java
index 61fa4e5..cd4dd99 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/registry/FunctionRegistryHolderTest.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
@@ -60,25 +60,24 @@ public class FunctionRegistryHolderTest {
   @Before
   public void setup() {
     resetRegistry();
-    fillInRegistry();
+    fillInRegistry(1);
   }
 
   @Test
   public void testVersion() {
     resetRegistry();
-    assertEquals("Initial version should be 0", 0, registryHolder.getVersion());
-    registryHolder.addJars(Maps.<String, List<FunctionHolder>>newHashMap());
-    assertEquals("Version should not change if no jars were added.", 0, registryHolder.getVersion());
-    registryHolder.removeJar("unknown.jar");
-    assertEquals("Version should not change if no jars were removed.", 0, registryHolder.getVersion());
-    fillInRegistry();
-    assertEquals("Version should have incremented by 1", 1, registryHolder.getVersion());
+    long expectedVersion = 0;
+    assertEquals("Initial version should be 0", expectedVersion, registryHolder.getVersion());
+    registryHolder.addJars(Maps.<String, List<FunctionHolder>>newHashMap(), ++expectedVersion);
+    assertEquals("Version can change if no jars were added.", expectedVersion, registryHolder.getVersion());
+    fillInRegistry(++expectedVersion);
+    assertEquals("Version should have incremented by 1", expectedVersion, registryHolder.getVersion());
     registryHolder.removeJar(built_in);
-    assertEquals("Version should have incremented by 1", 2, registryHolder.getVersion());
-    fillInRegistry();
-    assertEquals("Version should have incremented by 1", 3, registryHolder.getVersion());
-    fillInRegistry();
-    assertEquals("Version should have incremented by 1", 4, registryHolder.getVersion());
+    assertEquals("Version should have incremented by 1", expectedVersion, registryHolder.getVersion());
+    fillInRegistry(++expectedVersion);
+    assertEquals("Version should have incremented by 1", expectedVersion, registryHolder.getVersion());
+    fillInRegistry(++expectedVersion);
+    assertEquals("Version should have incremented by 1", expectedVersion, registryHolder.getVersion());
   }
 
   @Test
@@ -97,8 +96,9 @@ public class FunctionRegistryHolderTest {
       }
     }
 
-    registryHolder.addJars(newJars);
-    assertEquals("Version number should match", 1, registryHolder.getVersion());
+    long expectedVersion = 0;
+    registryHolder.addJars(newJars, ++expectedVersion);
+    assertEquals("Version number should match", expectedVersion, registryHolder.getVersion());
     compareTwoLists(jars, registryHolder.getAllJarNames());
     assertEquals(functionsSize, registryHolder.functionsSize());
     compareListMultimaps(functionsWithHolders, registryHolder.getAllFunctionsWithHolders());
@@ -120,16 +120,17 @@ public class FunctionRegistryHolderTest {
         functionsSize++;
       }
     }
-    registryHolder.addJars(newJars);
-    assertEquals("Version number should match", 1, registryHolder.getVersion());
+    long expectedVersion = 0;
+    registryHolder.addJars(newJars, ++expectedVersion);
+    assertEquals("Version number should match", expectedVersion, registryHolder.getVersion());
     compareTwoLists(jars, registryHolder.getAllJarNames());
     assertEquals(functionsSize, registryHolder.functionsSize());
     compareListMultimaps(functionsWithHolders, registryHolder.getAllFunctionsWithHolders());
     compareListMultimaps(functionsWithSignatures, registryHolder.getAllFunctionsWithSignatures());
 
     // adding the same jars should not cause adding duplicates, should override existing jars only
-    registryHolder.addJars(newJars);
-    assertEquals("Version number should match", 2, registryHolder.getVersion());
+    registryHolder.addJars(newJars, ++expectedVersion);
+    assertEquals("Version number should match", expectedVersion, registryHolder.getVersion());
     compareTwoLists(jars, registryHolder.getAllJarNames());
     assertEquals(functionsSize, registryHolder.functionsSize());
     compareListMultimaps(functionsWithHolders, registryHolder.getAllFunctionsWithHolders());
@@ -252,8 +253,8 @@ public class FunctionRegistryHolderTest {
     registryHolder = new FunctionRegistryHolder();
   }
 
-  private void fillInRegistry() {
-    registryHolder.addJars(newJars);
+  private void fillInRegistry(long version) {
+    registryHolder.addJars(newJars, version);
   }
 
   private <T> void compareListMultimaps(ListMultimap<String, T> lm1, ListMultimap<String, T> lm2) {

http://git-wip-us.apache.org/repos/asf/drill/blob/dcbcb94f/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
index 8b338af..2847696 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/ExpressionTreeMaterializerTest.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
@@ -45,7 +45,6 @@ import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.ExpressionTreeMaterializer;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.expr.fn.registry.RemoteFunctionRegistry;
-import org.apache.drill.exec.proto.UserBitShared.Registry;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
@@ -202,8 +201,8 @@ public class ExpressionTreeMaterializerTest extends ExecTest {
 
     new MockUp<RemoteFunctionRegistry>() {
       @Mock
-      Registry getRegistry() {
-        return Registry.getDefaultInstance();
+      long getRegistryVersion() {
+        return 0L;
       }
     };
 

http://git-wip-us.apache.org/repos/asf/drill/blob/dcbcb94f/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0-sources.jar
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0-sources.jar b/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0-sources.jar
new file mode 100644
index 0000000..f6b250e
Binary files /dev/null and b/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0-sources.jar differ

http://git-wip-us.apache.org/repos/asf/drill/blob/dcbcb94f/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0.jar
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0.jar b/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0.jar
new file mode 100644
index 0000000..4b5ef8b
Binary files /dev/null and b/exec/java-exec/src/test/resources/jars/DrillUDF-overloading-1.0.jar differ