You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by al...@apache.org on 2019/08/22 01:19:56 UTC

[asterixdb] branch master updated: [NO ISSUE][COMP] Meta() after group by

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

alsuliman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 28dd5e9  [NO ISSUE][COMP] Meta() after group by
28dd5e9 is described below

commit 28dd5e9bd9fe3fb5a1cbfa2b338ec4921b254727
Author: Ali Alsuliman <al...@gmail.com>
AuthorDate: Wed Aug 21 01:32:47 2019 -0700

    [NO ISSUE][COMP] Meta() after group by
    
    - user model changes: no
    - storage format changes: no
    - interface changes: yes
    
    Details:
    Using meta() after group-by requires meta() be assigned to
    a variable in a LET clause to expose meta().
    
    - added source location to IFunctionManager.lookupFunction()
    - added test cases
    
    Change-Id: I2b173e23352fa80ef57154f08910586c6ee196b8
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3528
    Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
 .../jobgen/QueryLogicalExpressionJobGen.java       |  6 ++--
 .../meta_after_gby/meta_after_gby.1.ddl.sqlpp}     | 40 ++++++++++++++++------
 .../meta_after_gby/meta_after_gby.2.update.sqlpp}  | 17 +++------
 .../meta_after_gby/meta_after_gby.3.query.sqlpp}   | 20 +++++------
 .../meta_after_gby/meta_after_gby.4.query.sqlpp}   | 19 +++++-----
 .../meta_after_gby/meta_after_gby.5.ddl.sqlpp}     | 15 +-------
 .../meta/meta_after_gby/meta_after_gby.3.adm       |  6 ++++
 .../test/resources/runtimets/testsuite_sqlpp.xml   |  6 ++++
 ...condaryCorrelatedTreeIndexOperationsHelper.java |  2 +-
 .../utils/SecondaryIndexOperationsHelper.java      |  6 ++--
 .../asterix/om/functions/IFunctionManager.java     |  4 ++-
 .../runtime/formats/NonTaggedDataFormat.java       | 13 ++++---
 .../asterix/runtime/functions/FunctionManager.java | 12 +++++--
 13 files changed, 92 insertions(+), 74 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
index 32f8001..0de905f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/jobgen/QueryLogicalExpressionJobGen.java
@@ -52,6 +52,7 @@ import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.ISerializedAggregateEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.base.IUnnestingEvaluatorFactory;
 import org.apache.hyracks.algebricks.runtime.evaluators.ColumnAccessEvalFactory;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 public class QueryLogicalExpressionJobGen implements ILogicalExpressionJobGen {
 
@@ -197,8 +198,9 @@ public class QueryLogicalExpressionJobGen implements ILogicalExpressionJobGen {
     private IFunctionDescriptor resolveFunction(AbstractFunctionCallExpression expr, IVariableTypeEnvironment env,
             JobGenContext context) throws AlgebricksException {
         FunctionIdentifier fnId = expr.getFunctionIdentifier();
-        IFunctionDescriptor fd = functionManager.lookupFunction(fnId);
-        fd.setSourceLocation(expr.getSourceLocation());
+        SourceLocation sourceLocation = expr.getSourceLocation();
+        IFunctionDescriptor fd = functionManager.lookupFunction(fnId, sourceLocation);
+        fd.setSourceLocation(sourceLocation);
         IFunctionTypeInferer fnTypeInfer = functionManager.lookupFunctionTypeInferer(fnId);
         if (fnTypeInfer != null) {
             CompilerProperties compilerProps = ((IApplicationContext) context.getAppContext()).getCompilerProperties();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.1.ddl.sqlpp
similarity index 56%
copy from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.1.ddl.sqlpp
index e1657ff..a80fc8a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.1.ddl.sqlpp
@@ -16,18 +16,36 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.functions;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+drop dataverse test if exists;
+create dataverse test;
+use test;
 
-/**
- * A registry of {@link IFunctionDescriptor} and {@link IFunctionTypeInferer}
- * for built-in functions
- */
-public interface IFunctionManager {
+create type DataType as open {
+id:int,
+name:string,
+age:int,
+hobby:string
+};
+
+create type MetaType as closed {
+id:int
+};
 
-    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
+create dataset DS(DataType) with meta(MetaType) primary key meta().id;
 
-    IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
-}
+create feed DsStream with {
+ "adapter-name" : "localfs",
+ "reader" : "localfs",
+ "parser" : "record-with-metadata",
+ "type-name" : "DataType",
+ "meta-type-name" : "MetaType",
+ "path" : "asterix_nc1://data/csv/people.csv",
+ "format" : "csv",
+ "delimiter" : ",",
+ "record-format" : "adm",
+ "record-index" : "1",
+ "key-indexes" : "0",
+ "key-indicators" : "1",
+ "header" : "false"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.2.update.sqlpp
similarity index 61%
copy from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.2.update.sqlpp
index e1657ff..6d41058 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.2.update.sqlpp
@@ -16,18 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.functions;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+use test;
 
-/**
- * A registry of {@link IFunctionDescriptor} and {@link IFunctionTypeInferer}
- * for built-in functions
- */
-public interface IFunctionManager {
-
-    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
-
-    IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
-}
+set `wait-for-completion-feed` "true";
+connect feed DsStream to dataset DS;
+start feed DsStream;
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.3.query.sqlpp
similarity index 61%
copy from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.3.query.sqlpp
index e1657ff..b317302 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.3.query.sqlpp
@@ -16,18 +16,16 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.functions;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-
-/**
- * A registry of {@link IFunctionDescriptor} and {@link IFunctionTypeInferer}
- * for built-in functions
+/*
+ * Description: use of meta() after group by requires meta() be assigned to a variable in a LET clause
+ * Result     : success
  */
-public interface IFunctionManager {
 
-    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
+USE test;
 
-    IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
-}
+FROM DS AS d
+LET meta_id = meta().id
+GROUP BY d.hobby AS hobby
+SELECT hobby, COUNT(meta_id) AS count
+ORDER BY hobby;
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.4.query.sqlpp
similarity index 61%
copy from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.4.query.sqlpp
index e1657ff..bb016f7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.4.query.sqlpp
@@ -16,18 +16,15 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.functions;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-
-/**
- * A registry of {@link IFunctionDescriptor} and {@link IFunctionTypeInferer}
- * for built-in functions
+/*
+ * Description: use of meta() after group by requires meta() be assigned to a variable in a LET clause
+ * Result     : failure
  */
-public interface IFunctionManager {
 
-    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
+USE test;
 
-    IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
-}
+FROM DS AS d
+GROUP BY d.hobby AS hobby
+SELECT hobby, COUNT(meta().id)
+ORDER BY hobby;
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.5.ddl.sqlpp
similarity index 61%
copy from asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.5.ddl.sqlpp
index e1657ff..f12a2b7 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/meta/meta_after_gby/meta_after_gby.5.ddl.sqlpp
@@ -16,18 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.om.functions;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-
-/**
- * A registry of {@link IFunctionDescriptor} and {@link IFunctionTypeInferer}
- * for built-in functions
- */
-public interface IFunctionManager {
-
-    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
-
-    IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
-}
+drop dataverse test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_after_gby/meta_after_gby.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_after_gby/meta_after_gby.3.adm
new file mode 100644
index 0000000..bf1932c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/meta/meta_after_gby/meta_after_gby.3.adm
@@ -0,0 +1,6 @@
+{ "hobby": "basketball", "count": 1 }
+{ "hobby": "bowling", "count": 1 }
+{ "hobby": "hiking", "count": 2 }
+{ "hobby": "reading", "count": 2 }
+{ "hobby": "soccer", "count": 1 }
+{ "hobby": "tennis", "count": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 17f4aee..ccd8f5a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -11358,6 +11358,12 @@
         <output-dir compare="Text">resolving_pk_with_meta</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="meta">
+      <compilation-unit name="meta_after_gby">
+        <output-dir compare="Text">meta_after_gby</output-dir>
+        <expected-error>Compilation error: Inappropriate use of function 'meta'. For example, after GROUP BY (in line 29, at column 21)</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="big-object">
     <test-case FilePath="big-object">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
index b044293..06df020 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryCorrelatedTreeIndexOperationsHelper.java
@@ -192,7 +192,7 @@ public abstract class SecondaryCorrelatedTreeIndexOperationsHelper extends Secon
         IScalarEvaluatorFactory[] castEvalFact =
                 new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(recordIdx) };
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
-        sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
+        sefs[0] = createCastFunction(strictCast, sourceLoc).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
         castAssign.setSourceLocation(sourceLoc);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
index 318814c..6bcc039 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/SecondaryIndexOperationsHelper.java
@@ -327,16 +327,16 @@ public abstract class SecondaryIndexOperationsHelper {
         IScalarEvaluatorFactory[] castEvalFact =
                 new IScalarEvaluatorFactory[] { new ColumnAccessEvalFactory(recordIdx) };
         IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[1];
-        sefs[0] = createCastFunction(strictCast).createEvaluatorFactory(castEvalFact);
+        sefs[0] = createCastFunction(strictCast, sourceLoc).createEvaluatorFactory(castEvalFact);
         AssignRuntimeFactory castAssign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
         castAssign.setSourceLocation(sourceLoc);
         return new AlgebricksMetaOperatorDescriptor(spec, 1, 1, new IPushRuntimeFactory[] { castAssign },
                 new RecordDescriptor[] { enforcedRecDesc });
     }
 
-    protected IFunctionDescriptor createCastFunction(boolean strictCast) throws AlgebricksException {
+    IFunctionDescriptor createCastFunction(boolean strictCast, SourceLocation sourceLoc) throws AlgebricksException {
         IFunctionDescriptor castFuncDesc = metadataProvider.getFunctionManager()
-                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX);
+                .lookupFunction(strictCast ? BuiltinFunctions.CAST_TYPE : BuiltinFunctions.CAST_TYPE_LAX, sourceLoc);
         castFuncDesc.setSourceLocation(sourceLoc);
         castFuncDesc.setImmutableStates(enforcedItemType, itemType);
         return castFuncDesc;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
index e1657ff..0d0b3c6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IFunctionManager.java
@@ -20,6 +20,7 @@ package org.apache.asterix.om.functions;
 
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * A registry of {@link IFunctionDescriptor} and {@link IFunctionTypeInferer}
@@ -27,7 +28,8 @@ import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
  */
 public interface IFunctionManager {
 
-    IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException;
+    IFunctionDescriptor lookupFunction(FunctionIdentifier fid, SourceLocation sourceLocation)
+            throws AlgebricksException;
 
     IFunctionTypeInferer lookupFunctionTypeInferer(FunctionIdentifier fid);
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
index 8b25e06..0db1821 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -141,7 +141,8 @@ public class NonTaggedDataFormat implements IDataFormat {
                     }
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                    IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    IFunctionDescriptor fDesc =
+                            functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, sourceLoc);
                     fDesc.setSourceLocation(sourceLoc);
                     fDesc.setImmutableStates(recType);
                     return fDesc.createEvaluatorFactory(
@@ -158,7 +159,8 @@ public class NonTaggedDataFormat implements IDataFormat {
                 }
                 IScalarEvaluatorFactory fldNameEvalFactory =
                         new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME);
+                IFunctionDescriptor fDesc =
+                        functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_NAME, sourceLoc);
                 fDesc.setSourceLocation(sourceLoc);
                 return fDesc.createEvaluatorFactory(
                         new IScalarEvaluatorFactory[] { recordEvalFactory, fldNameEvalFactory });
@@ -166,7 +168,7 @@ public class NonTaggedDataFormat implements IDataFormat {
         }
 
         if (fldName.size() > 1) {
-            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED, sourceLoc);
             fDesc.setSourceLocation(sourceLoc);
             fDesc.setImmutableStates(recType, fldName);
             return fDesc.createEvaluatorFactory(new IScalarEvaluatorFactory[] { recordEvalFactory });
@@ -241,7 +243,8 @@ public class NonTaggedDataFormat implements IDataFormat {
                     }
                     IScalarEvaluatorFactory fldIndexEvalFactory =
                             new ConstantEvalFactory(Arrays.copyOf(abvs.getByteArray(), abvs.getLength()));
-                    IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+                    IFunctionDescriptor fDesc =
+                            functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_BY_INDEX, sourceLoc);
                     fDesc.setSourceLocation(sourceLoc);
                     fDesc.setImmutableStates(recType);
                     IScalarEvaluatorFactory evalFactory = fDesc.createEvaluatorFactory(
@@ -267,7 +270,7 @@ public class NonTaggedDataFormat implements IDataFormat {
             } catch (HyracksDataException e) {
                 throw new AlgebricksException(e);
             }
-            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED);
+            IFunctionDescriptor fDesc = functionManager.lookupFunction(BuiltinFunctions.FIELD_ACCESS_NESTED, sourceLoc);
             fDesc.setSourceLocation(sourceLoc);
             fDesc.setImmutableStates(recType, fldName);
             IScalarEvaluatorFactory evalFactory =
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
index d5a6559..7158558 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionManager.java
@@ -22,6 +22,9 @@ package org.apache.asterix.runtime.functions;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionManager;
@@ -29,6 +32,7 @@ import org.apache.asterix.om.functions.IFunctionTypeInferer;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
 
 /**
  * Default implementation of {@link IFunctionManager}.
@@ -57,11 +61,15 @@ public final class FunctionManager implements IFunctionManager {
     }
 
     @Override
-    public IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException {
+    public IFunctionDescriptor lookupFunction(FunctionIdentifier fid, SourceLocation src) throws AlgebricksException {
         Pair<FunctionIdentifier, Integer> key = new Pair<>(fid, fid.getArity());
         IFunctionDescriptorFactory factory = functions.get(key);
         if (factory == null) {
-            throw new AlgebricksException("Inappropriate use of function " + "'" + fid.getName() + "'");
+            String msg = "Inappropriate use of function '" + fid.getName() + "'";
+            if (fid.equals(BuiltinFunctions.META)) {
+                msg = msg + ". For example, after GROUP BY";
+            }
+            throw AsterixException.create(ErrorCode.COMPILATION_ERROR, src, msg);
         }
         return factory.createFunctionDescriptor();
     }