You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by st...@apache.org on 2020/03/02 13:59:47 UTC

[phoenix] branch 4.x-HBase-1.4 updated: PHOENIX-4866 UDFs get error: org.apache.phoenix.schema.FunctionNotFoundException: ERROR 6001 (42F01): Function undefined

This is an automated email from the ASF dual-hosted git repository.

stoty pushed a commit to branch 4.x-HBase-1.4
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x-HBase-1.4 by this push:
     new 3a1abcf  PHOENIX-4866 UDFs get error: org.apache.phoenix.schema.FunctionNotFoundException: ERROR 6001 (42F01): Function undefined
3a1abcf is described below

commit 3a1abcf09ccee66f70b28661bbade2ac684d3121
Author: Richard Antal <an...@gmail.com>
AuthorDate: Tue Feb 25 13:07:59 2020 +0100

    PHOENIX-4866 UDFs get error: org.apache.phoenix.schema.FunctionNotFoundException: ERROR 6001 (42F01): Function undefined
---
 .../phoenix/end2end/UserDefinedFunctionsIT.java    |   5 +
 .../org/apache/phoenix/compile/FromCompiler.java   | 271 ++++++++++++---------
 2 files changed, 167 insertions(+), 109 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index 75e9afe..92b253a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -381,6 +381,11 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
         assertTrue(rs.next());
         assertEquals("oof", rs.getString(1));        
         assertFalse(rs.next());
+        rs = stmt.executeQuery("select myreverse2('abc'), myreverse2('aba')");
+        assertTrue(rs.next());
+        assertEquals("cba", rs.getString(1));
+        assertEquals("aba", rs.getString(2));
+        assertFalse(rs.next());
         rs = stmt.executeQuery("select * from t where myreverse2(firstname)='oof'");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index 98a58a9..abdf662 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -222,7 +222,7 @@ public class FromCompiler {
     		throws SQLException {
     	TableNode fromNode = statement.getFrom();
     	if (fromNode == null)
-    	    return EMPTY_TABLE_RESOLVER;
+            return new ColumnResolverWithUDF(connection, 1, true, statement.getUdfParseNodes());
         if (fromNode instanceof NamedTableNode)
             return new SingleTableColumnResolver(connection, (NamedTableNode) fromNode, true, 1, statement.getUdfParseNodes(), alwaysHitServer, mutatingTableName);
 
@@ -540,41 +540,189 @@ public class FromCompiler {
         }
     }
 
-    private static abstract class BaseColumnResolver implements ColumnResolver {
+    private static class ColumnResolverWithUDF implements ColumnResolver {
         protected final PhoenixConnection connection;
         protected final MetaDataClient client;
         // Fudge factor to add to current time we calculate. We need this when we do a SELECT
         // on Windows because the millis timestamp granularity is so bad we sometimes won't
         // get the data back that we just upsert.
-        private final int tsAddition;
+        protected final int tsAddition;
         protected final Map<String, PFunction> functionMap;
         protected List<PFunction> functions;
         //PHOENIX-3823 : Force update cache when mutating table and select table are same
         //(UpsertSelect or Delete with select on same table)
-        protected TableName mutatingTableName = null;
 
-        private BaseColumnResolver(PhoenixConnection connection, int tsAddition, TableName mutatingTableName) {
+        private ColumnResolverWithUDF(PhoenixConnection connection, int tsAddition,
+                                      boolean updateCacheImmediately, Map<String,
+                UDFParseNode> udfParseNodes) throws SQLException {
             this.connection = connection;
             this.client = connection == null ? null : new MetaDataClient(connection);
             this.tsAddition = tsAddition;
             functionMap = new HashMap<String, PFunction>(1);
-            this.functions = Collections.<PFunction>emptyList();
-            this.mutatingTableName = mutatingTableName;
+            if (udfParseNodes.isEmpty()) {
+                functions = Collections.<PFunction> emptyList();
+            } else {
+                functions = createFunctionRef(new ArrayList<String>(udfParseNodes.keySet()),
+                        updateCacheImmediately);
+                for (PFunction function : functions) {
+                    functionMap.put(function.getFunctionName(), function);
+                }
+            }
         }
 
-        private BaseColumnResolver(PhoenixConnection connection, int tsAddition, boolean updateCacheImmediately, Map<String, UDFParseNode> udfParseNodes, TableName mutatingTableName) throws SQLException {
-        	this.connection = connection;
+        private ColumnResolverWithUDF(PhoenixConnection connection, int tsAddition) {
+            this.connection = connection;
             this.client = connection == null ? null : new MetaDataClient(connection);
             this.tsAddition = tsAddition;
             functionMap = new HashMap<String, PFunction>(1);
-            if (udfParseNodes.isEmpty()) {
-                functions = Collections.<PFunction> emptyList();
+            this.functions = Collections.<PFunction>emptyList();
+        }
+
+        @Override
+        public List<PFunction> getFunctions() {
+            return functions;
+        }
+
+        private List<PFunction> createFunctionRef(List<String> functionNames,
+                                                  boolean updateCacheImmediately)
+                throws SQLException {
+            long timeStamp = QueryConstants.UNSET_TIMESTAMP;
+            int numFunctions = functionNames.size();
+            List<PFunction> functionsFound = new ArrayList<PFunction>(functionNames.size());
+            if (updateCacheImmediately || connection.getAutoCommit()) {
+                getFunctionFromCache(functionNames, functionsFound, true);
+                if(functionNames.isEmpty()) {
+                    return functionsFound;
+                }
+                MetaDataMutationResult result = client.updateCache(functionNames);
+                timeStamp = result.getMutationTime();
+                functionsFound = result.getFunctions();
+                if(functionNames.size() != functionsFound.size()){
+                    throw new FunctionNotFoundException("Some of the functions in " +
+                            functionNames.toString()+" are not found");
+                }
             } else {
-                functions = createFunctionRef(new ArrayList<String>(udfParseNodes.keySet()), updateCacheImmediately);
-                for (PFunction function : functions) {
-                    functionMap.put(function.getFunctionName(), function);
+                getFunctionFromCache(functionNames, functionsFound, false);
+                // We always attempt to update the cache in the event of a FunctionNotFoundException
+                MetaDataMutationResult result = null;
+                if (!functionNames.isEmpty()) {
+                    result = client.updateCache(functionNames);
+                }
+                if(result!=null) {
+                    if (!result.getFunctions().isEmpty()) {
+                        functionsFound.addAll(result.getFunctions());
+                    }
+                    if(result.wasUpdated()) {
+                        timeStamp = result.getMutationTime();
+                    }
+                }
+                if (functionsFound.size()!=numFunctions) {
+                    throw new FunctionNotFoundException("Some of the functions in " +
+                            functionNames.toString()+" are not found", timeStamp);
                 }
             }
+            if (timeStamp != QueryConstants.UNSET_TIMESTAMP) {
+                timeStamp += tsAddition;
+            }
+
+            if (LOGGER.isDebugEnabled() && timeStamp != QueryConstants.UNSET_TIMESTAMP) {
+                LOGGER.debug(LogUtil.addCustomAnnotations("Re-resolved stale function " +
+                        functionNames.toString() + "at timestamp " + timeStamp, connection));
+            }
+            return functionsFound;
+        }
+
+        private void getFunctionFromCache(List<String> functionNames,
+                                          List<PFunction> functionsFound,
+                                          boolean getOnlyTemporyFunctions) {
+            Iterator<String> iterator = functionNames.iterator();
+            while(iterator.hasNext()) {
+                PFunction function = null;
+                String functionName = iterator.next();
+                try {
+                    function = connection.getMetaDataCache().getFunction(
+                            new PTableKey(connection.getTenantId(), functionName));
+                } catch (FunctionNotFoundException e1) {
+                    if (connection.getTenantId() != null) { // Check with null tenantId next
+                        try {
+                            function = connection.getMetaDataCache().getFunction(
+                                    new PTableKey(null, functionName));
+                        } catch (FunctionNotFoundException ignored) {
+                        }
+                    }
+                }
+                if (function != null) {
+                    if (getOnlyTemporyFunctions) {
+                        if (function.isTemporaryFunction()) {
+                            functionsFound.add(function);
+                            iterator.remove();
+                        }
+                    } else {
+                        functionsFound.add(function);
+                        iterator.remove();
+                    }
+                }
+            }
+        }
+
+        @Override
+        public PFunction resolveFunction(String functionName) throws SQLException {
+            PFunction function = functionMap.get(functionName);
+            if(function == null) {
+                throw new FunctionNotFoundException(functionName);
+            }
+            return function;
+        }
+
+        @Override
+        public boolean hasUDFs() {
+            return !functions.isEmpty();
+        }
+
+        @Override
+        public List<TableRef> getTables() {
+            return Collections.singletonList(TableRef.EMPTY_TABLE_REF);
+        }
+
+
+        @Override
+        public TableRef resolveTable(String schemaName, String tableName)
+                throws SQLException {
+            throw new TableNotFoundException(schemaName, tableName);
+        }
+
+        @Override
+        public ColumnRef resolveColumn(String schemaName, String tableName, String colName)
+                throws SQLException {
+            throw new ColumnNotFoundException(schemaName, tableName, null, colName);
+        }
+
+        @Override
+        public PSchema resolveSchema(String schemaName) throws SQLException {
+            throw new SchemaNotFoundException(schemaName);
+        }
+
+        @Override
+        public List<PSchema> getSchemas() {
+            return Collections.emptyList();
+        }
+
+    }
+
+    private static abstract class BaseColumnResolver extends ColumnResolverWithUDF {
+        protected TableName mutatingTableName = null;
+
+        private BaseColumnResolver(PhoenixConnection connection, int tsAddition,
+                                   TableName mutatingTableName) {
+            super(connection, tsAddition);
+            this.mutatingTableName = mutatingTableName;
+        }
+
+        private BaseColumnResolver(PhoenixConnection connection, int tsAddition,
+                                   boolean updateCacheImmediately,
+                                   Map<String, UDFParseNode> udfParseNodes,
+                                   TableName mutatingTableName) throws SQLException {
+            super(connection, tsAddition, updateCacheImmediately, udfParseNodes);
             this.mutatingTableName = mutatingTableName;
         }
 
@@ -669,87 +817,6 @@ public class FromCompiler {
             return tableRef;
         }
 
-        @Override
-        public List<PFunction> getFunctions() {
-            return functions;
-        }
-
-        private List<PFunction> createFunctionRef(List<String> functionNames, boolean updateCacheImmediately) throws SQLException {
-            long timeStamp = QueryConstants.UNSET_TIMESTAMP;
-            int numFunctions = functionNames.size();
-            List<PFunction> functionsFound = new ArrayList<PFunction>(functionNames.size());
-            if (updateCacheImmediately || connection.getAutoCommit()) {
-                getFunctionFromCache(functionNames, functionsFound, true);
-                if(functionNames.isEmpty()) {
-                    return functionsFound;
-                }
-                MetaDataMutationResult result = client.updateCache(functionNames);
-                timeStamp = result.getMutationTime();
-                functionsFound = result.getFunctions();
-                if(functionNames.size() != functionsFound.size()){
-                    throw new FunctionNotFoundException("Some of the functions in "+functionNames.toString()+" are not found");
-                }
-            } else {
-                getFunctionFromCache(functionNames, functionsFound, false);
-                // We always attempt to update the cache in the event of a FunctionNotFoundException
-                MetaDataMutationResult result = null;
-                if (!functionNames.isEmpty()) {
-                    result = client.updateCache(functionNames);
-                }
-                if(result!=null) {
-                    if (!result.getFunctions().isEmpty()) {
-                        functionsFound.addAll(result.getFunctions());
-                    }
-                    if(result.wasUpdated()) {
-                        timeStamp = result.getMutationTime();
-                    }
-                }
-                if (functionsFound.size()!=numFunctions) {
-                    throw new FunctionNotFoundException("Some of the functions in "+functionNames.toString()+" are not found", timeStamp);
-                }
-            }
-            if (timeStamp != QueryConstants.UNSET_TIMESTAMP) {
-                timeStamp += tsAddition;
-            }
-            
-            if (LOGGER.isDebugEnabled() && timeStamp != QueryConstants.UNSET_TIMESTAMP) {
-                LOGGER.debug(LogUtil.addCustomAnnotations(
-                        "Re-resolved stale function " + functionNames.toString() +
-                                "at timestamp " + timeStamp, connection));
-            }
-            return functionsFound;
-        }
-
-        private void getFunctionFromCache(List<String> functionNames,
-                List<PFunction> functionsFound, boolean getOnlyTemporyFunctions) {
-            Iterator<String> iterator = functionNames.iterator();
-            while(iterator.hasNext()) {
-                PFunction function = null;
-                String functionName = iterator.next();
-                try {
-                    function = connection.getMetaDataCache().getFunction(new PTableKey(connection.getTenantId(), functionName));
-                } catch (FunctionNotFoundException e1) {
-                    if (connection.getTenantId() != null) { // Check with null tenantId next
-                        try {
-                            function = connection.getMetaDataCache().getFunction(new PTableKey(null, functionName));
-                        } catch (FunctionNotFoundException e2) {
-                        }
-                    }
-                }
-                if (function != null) {
-                    if (getOnlyTemporyFunctions) {
-                        if (function.isTemporaryFunction()) {
-                            functionsFound.add(function);
-                            iterator.remove();
-                        }
-                    } else {
-                        functionsFound.add(function);
-                        iterator.remove();
-                    }
-                }
-            }
-        }
-
         protected PTable addDynamicColumns(List<ColumnDef> dynColumns, PTable theTable)
                 throws SQLException {
             if (!dynColumns.isEmpty()) {
@@ -779,20 +846,6 @@ public class FromCompiler {
             }
             return theTable;
         }
-        
-        @Override
-        public PFunction resolveFunction(String functionName) throws SQLException {
-            PFunction function = functionMap.get(functionName);
-            if(function == null) {
-                throw new FunctionNotFoundException(functionName);
-            }
-            return function;
-        }
-
-        @Override
-        public boolean hasUDFs() {
-            return !functions.isEmpty();
-        }
     }
 
     private static class MultiTableColumnResolver extends BaseColumnResolver implements TableNodeVisitor<Void> {