You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2022/11/16 16:36:30 UTC

[calcite] branch main updated (a0ce327511 -> 406c913b80)

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

jhyde pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git


    from a0ce327511 [CALCITE-5310] JSON_OBJECT in scalar sub-query throws AssertionError
     add c40aa4ce7e Quidem: Allow CREATE VIEW in 'scott' connection
     new 406c913b80 [CALCITE-5105] Add MEASURE type and AGGREGATE aggregate function

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../apache/calcite/rel/hint/HintStrategyTable.java | 13 ----
 .../calcite/rel/type/RelDataTypeFactory.java       |  8 ++
 .../apache/calcite/runtime/CalciteResource.java    |  9 +++
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  3 +
 .../main/java/org/apache/calcite/sql/SqlNode.java  |  7 +-
 .../java/org/apache/calcite/sql/SqlNodeList.java   | 26 ++-----
 .../org/apache/calcite/sql/SqlRowTypeNameSpec.java | 15 +---
 .../main/java/org/apache/calcite/sql/SqlUtil.java  | 13 ++++
 .../org/apache/calcite/sql/fun/SqlLibrary.java     |  2 +
 .../calcite/sql/fun/SqlLibraryOperators.java       | 12 +++
 ...OperandTypeInference.java => ApplySqlType.java} | 45 +++++++----
 ...SqlOperandMetadata.java => MeasureSqlType.java} | 29 ++++---
 .../org/apache/calcite/sql/type/OperandTypes.java  | 26 +++++++
 .../calcite/sql/type/SqlTypeFactoryImpl.java       |  5 ++
 .../org/apache/calcite/sql/type/SqlTypeName.java   |  1 +
 .../apache/calcite/sql/type/SqlTypeTransforms.java | 12 +++
 .../org/apache/calcite/sql/util/SqlVisitor.java    |  5 ++
 .../apache/calcite/sql/validate/AggChecker.java    | 38 +++++++--
 .../sql/validate/AggregatingSelectScope.java       | 50 ++++++------
 .../calcite/sql/validate/DelegatingScope.java      | 21 ++++-
 .../apache/calcite/sql/validate/SqlValidator.java  | 10 +++
 .../calcite/sql/validate/SqlValidatorImpl.java     | 10 +++
 .../calcite/sql/validate/SqlValidatorScope.java    | 22 ++++++
 .../calcite/sql/validate/SqlValidatorUtil.java     | 19 ++++-
 .../main/java/org/apache/calcite/util/Litmus.java  | 53 +++++++------
 .../calcite/runtime/CalciteResource.properties     |  3 +
 core/src/test/codegen/config.fmpp                  |  2 +
 core/src/test/codegen/includes/parserImpls.ftl     | 15 ++++
 .../parserextensiontesting/SqlCreateTable.java     |  8 +-
 .../org/apache/calcite/test/CoreQuidemTest.java    |  8 +-
 .../apache/calcite/test/ExtensionDdlExecutor.java  | 80 ++++++++++++++++++-
 .../apache/calcite/test/SqlToRelConverterTest.java | 18 +++++
 .../org/apache/calcite/test/SqlValidatorTest.java  | 90 ++++++++++++++++++++++
 .../apache/calcite/test/SqlToRelConverterTest.xml  | 14 ++++
 .../src/test/resources/sql/measure.iq              | 31 ++++++--
 core/src/test/resources/sql/misc.iq                | 18 +++++
 .../apache/calcite/server/ServerDdlExecutor.java   |  5 +-
 site/_docs/reference.md                            |  8 ++
 .../org/apache/calcite/test/SqlToRelTestBase.java  |  2 +-
 .../test/catalog/MockCatalogReaderExtended.java    | 19 ++++-
 40 files changed, 625 insertions(+), 150 deletions(-)
 copy core/src/main/java/org/apache/calcite/sql/type/{SqlOperandTypeInference.java => ApplySqlType.java} (51%)
 copy core/src/main/java/org/apache/calcite/sql/type/{SqlOperandMetadata.java => MeasureSqlType.java} (58%)
 copy babel/src/test/resources/sql/dummy.iq => core/src/test/resources/sql/measure.iq (65%)
 mode change 100755 => 100644


[calcite] 01/01: [CALCITE-5105] Add MEASURE type and AGGREGATE aggregate function

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhyde pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 406c913b808b3234464d8c81d7352c4040dd281a
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon Oct 25 17:06:56 2021 -0700

    [CALCITE-5105] Add MEASURE type and AGGREGATE aggregate function
    
    The MEASURE type is internal. A RexNode expression that
    contains a measure and evaluates to an INTEGER will have the
    type MEASURE<INTEGER>, using a parameterized SQL type
    similar to ARRAY<INTEGER>. But the same measure column, as
    seen from SQL, will just have type INTEGER. The
    parameterized type helps us keep things straight in the
    relational algebra as we apply planner rules.
    
    The AGGREGATE function belongs to the new CALCITE function
    library. To use it, add 'lib=calcite' to your connect string.
    
    Add a new validator configuration parameter,
    `boolean SqlValidator.Config.nakedMeasures()`. The query
    
      SELECT deptno, AGGREGATE(avg_sal)
      FROM emp
      GROUP BY deptno
    
    is valid if `avg_sal` is a measure. If `nakedMeasures` is
    true, then the following query is a valid shorthand for it:
    
      SELECT deptno, avg_sal
      FROM emp
      GROUP BY deptno
    
    In the long term, we would like people to feel comfortable
    using the latter form. Measures are not necessarily aggregate
    functions, but are just expressions whose value depends on
    their context (the current GROUP BY key in an aggregate
    query, or the current row in a regular query). And we will
    generalize measures to analytic expressions, which are not
    necessarily just references to measure columns.
    
    But in the short term, setting the `nakedMeasures` flag to
    false provides a level of comfort to people (and tools that
    generate SQL) who think of measures as aggregate functions,
    and think that measures should only be used in `GROUP BY`
    queries.
    
    Extend mock catalog with a table that has measure columns.
    
    Add a new Quidem test, measure.iq. It is disabled because we
    don't yet have the means to create measure columns in queries
    (or views). That is to come in [CALCITE-4496].
    
    Close apache/calcite#2965
---
 .../apache/calcite/rel/hint/HintStrategyTable.java | 13 ----
 .../calcite/rel/type/RelDataTypeFactory.java       |  8 ++
 .../apache/calcite/runtime/CalciteResource.java    |  9 +++
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  3 +
 .../main/java/org/apache/calcite/sql/SqlNode.java  |  7 +-
 .../java/org/apache/calcite/sql/SqlNodeList.java   | 26 ++-----
 .../org/apache/calcite/sql/SqlRowTypeNameSpec.java | 15 +---
 .../main/java/org/apache/calcite/sql/SqlUtil.java  | 13 ++++
 .../org/apache/calcite/sql/fun/SqlLibrary.java     |  2 +
 .../calcite/sql/fun/SqlLibraryOperators.java       | 12 +++
 .../org/apache/calcite/sql/type/ApplySqlType.java  | 56 ++++++++++++++
 .../apache/calcite/sql/type/MeasureSqlType.java    | 40 ++++++++++
 .../org/apache/calcite/sql/type/OperandTypes.java  | 26 +++++++
 .../calcite/sql/type/SqlTypeFactoryImpl.java       |  5 ++
 .../org/apache/calcite/sql/type/SqlTypeName.java   |  1 +
 .../apache/calcite/sql/type/SqlTypeTransforms.java | 12 +++
 .../org/apache/calcite/sql/util/SqlVisitor.java    |  5 ++
 .../apache/calcite/sql/validate/AggChecker.java    | 38 +++++++--
 .../sql/validate/AggregatingSelectScope.java       | 50 ++++++------
 .../calcite/sql/validate/DelegatingScope.java      | 21 ++++-
 .../apache/calcite/sql/validate/SqlValidator.java  | 10 +++
 .../calcite/sql/validate/SqlValidatorImpl.java     | 10 +++
 .../calcite/sql/validate/SqlValidatorScope.java    | 22 ++++++
 .../calcite/sql/validate/SqlValidatorUtil.java     | 19 ++++-
 .../main/java/org/apache/calcite/util/Litmus.java  | 53 +++++++------
 .../calcite/runtime/CalciteResource.properties     |  3 +
 .../apache/calcite/test/SqlToRelConverterTest.java | 18 +++++
 .../org/apache/calcite/test/SqlValidatorTest.java  | 90 ++++++++++++++++++++++
 .../apache/calcite/test/SqlToRelConverterTest.xml  | 14 ++++
 core/src/test/resources/sql/measure.iq             | 44 +++++++++++
 site/_docs/reference.md                            |  8 ++
 .../org/apache/calcite/test/SqlToRelTestBase.java  |  2 +-
 .../test/catalog/MockCatalogReaderExtended.java    | 19 ++++-
 33 files changed, 568 insertions(+), 106 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/hint/HintStrategyTable.java b/core/src/main/java/org/apache/calcite/rel/hint/HintStrategyTable.java
index 3c91690386..f2ec1a1163 100644
--- a/core/src/main/java/org/apache/calcite/rel/hint/HintStrategyTable.java
+++ b/core/src/main/java/org/apache/calcite/rel/hint/HintStrategyTable.java
@@ -233,18 +233,5 @@ public class HintStrategyTable {
       LOGGER.warn(requireNonNull(message, "message"), args);
       return false;
     }
-
-    @Override public boolean succeed() {
-      return true;
-    }
-
-    @Override public boolean check(boolean condition, @Nullable String message,
-        @Nullable Object... args) {
-      if (condition) {
-        return succeed();
-      } else {
-        return fail(message, args);
-      }
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
index 2000ccfa17..7ed4a5c065 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactory.java
@@ -132,6 +132,14 @@ public interface RelDataTypeFactory {
       RelDataType keyType,
       RelDataType valueType);
 
+  /**
+   * Creates a measure type.
+   *
+   * @param valueType type of the values of the measure
+   * @return canonical measure type descriptor
+   */
+  RelDataType createMeasureType(RelDataType valueType);
+
   /**
    * Creates a multiset type. Multisets are unordered collections of elements.
    *
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index de7c8fa423..88245302bc 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -326,6 +326,9 @@ public interface CalciteResource {
   @BaseMessage("OVER clause is necessary for window functions")
   ExInst<SqlValidatorException> absentOverClause();
 
+  @BaseMessage("Argument to function ''{0}'' must be a measure")
+  ExInst<SqlValidatorException> argumentMustBeMeasure(String functionName);
+
   @BaseMessage("Window ''{0}'' not found")
   ExInst<SqlValidatorException> windowNotFound(String a0);
 
@@ -356,6 +359,12 @@ public interface CalciteResource {
   @BaseMessage("Aggregate expressions cannot be nested")
   ExInst<SqlValidatorException> nestedAggIllegal();
 
+  @BaseMessage("Measure expressions can only occur within AGGREGATE function")
+  ExInst<SqlValidatorException> measureIllegal();
+
+  @BaseMessage("Measure expressions can only occur within a GROUP BY query")
+  ExInst<SqlValidatorException> measureMustBeInAggregateQuery();
+
   @BaseMessage("FILTER must not contain aggregate expression")
   ExInst<SqlValidatorException> aggregateInFilterIllegal();
 
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 9dff2716d8..4d7b8b974e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -891,6 +891,9 @@ public enum SqlKind {
   /** The {@code SINGLE_VALUE} aggregate function. */
   SINGLE_VALUE,
 
+  /** The {@code AGGREGATE} aggregate function. */
+  AGGREGATE_FN,
+
   /** The {@code BIT_AND} aggregate function. */
   BIT_AND,
 
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index 5626084d4f..9baa97ea2c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -341,9 +341,10 @@ public abstract class SqlNode implements Cloneable {
     return SqlMonotonicity.NOT_MONOTONIC;
   }
 
-  /** Returns whether two lists of operands are equal. */
-  public static boolean equalDeep(List<SqlNode> operands0,
-      List<SqlNode> operands1, Litmus litmus) {
+  /** Returns whether two lists of operands are equal, comparing using
+   * {@link SqlNode#equalsDeep(SqlNode, Litmus)}. */
+  public static boolean equalDeep(List<? extends @Nullable SqlNode> operands0,
+      List<? extends @Nullable SqlNode> operands1, Litmus litmus) {
     if (operands0.size() != operands1.size()) {
       return litmus.fail(null);
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
index 98ef092d47..dfa0486e30 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
@@ -34,6 +34,7 @@ import java.util.List;
 import java.util.ListIterator;
 import java.util.Objects;
 import java.util.RandomAccess;
+import java.util.function.Consumer;
 
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 
@@ -129,6 +130,11 @@ public class SqlNodeList extends SqlNode implements List<SqlNode>, RandomAccess
     return list.size();
   }
 
+  @Override public void forEach(Consumer<? super SqlNode> action) {
+    //noinspection RedundantCast
+    ((List<SqlNode>) list).forEach(action);
+  }
+
   @SuppressWarnings("return.type.incompatible")
   @Override public Iterator</*Nullable*/ SqlNode> iterator() {
     return list.iterator();
@@ -271,24 +277,8 @@ public class SqlNodeList extends SqlNode implements List<SqlNode>, RandomAccess
       return litmus.fail("{} != {}", this, node);
     }
     SqlNodeList that = (SqlNodeList) node;
-    if (this.size() != that.size()) {
-      return litmus.fail("{} != {}", this, node);
-    }
-    for (int i = 0; i < list.size(); i++) {
-      SqlNode thisChild = list.get(i);
-      final SqlNode thatChild = that.list.get(i);
-      if (thisChild == null) {
-        if (thatChild == null) {
-          continue;
-        } else {
-          return litmus.fail(null);
-        }
-      }
-      if (!thisChild.equalsDeep(thatChild, litmus)) {
-        return litmus.fail(null);
-      }
-    }
-    return litmus.succeed();
+    return SqlNode.equalDeep(list, that.list,
+        litmus.withMessageArgs("{} != {}", this, node));
   }
 
   public static boolean isEmptyList(final SqlNode node) {
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlRowTypeNameSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlRowTypeNameSpec.java
index 909b761b10..2e0d8ef02f 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlRowTypeNameSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlRowTypeNameSpec.java
@@ -106,22 +106,13 @@ public class SqlRowTypeNameSpec extends SqlTypeNameSpec {
       return litmus.fail("{} != {}", this, node);
     }
     SqlRowTypeNameSpec that = (SqlRowTypeNameSpec) node;
-    if (this.fieldNames.size() != that.fieldNames.size()) {
+    if (!SqlNode.equalDeep(this.fieldNames, that.fieldNames,
+        litmus.withMessageArgs("{} != {}", this, node))) {
       return litmus.fail("{} != {}", this, node);
     }
-    for (int i = 0; i < fieldNames.size(); i++) {
-      if (!this.fieldNames.get(i).equalsDeep(that.fieldNames.get(i), litmus)) {
-        return litmus.fail("{} != {}", this, node);
-      }
-    }
-    if (this.fieldTypes.size() != that.fieldTypes.size()) {
+    if (!this.fieldTypes.equals(that.fieldTypes)) {
       return litmus.fail("{} != {}", this, node);
     }
-    for (int i = 0; i < fieldTypes.size(); i++) {
-      if (!this.fieldTypes.get(i).equals(that.fieldTypes.get(i))) {
-        return litmus.fail("{} != {}", this, node);
-      }
-    }
     return litmus.succeed();
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 5ee6e560cb..37c5afbe15 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -42,6 +42,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.BarfingInvocationHandler;
 import org.apache.calcite.util.ConversionUtil;
 import org.apache.calcite.util.Glossary;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
@@ -161,6 +162,18 @@ public abstract class SqlUtil {
     return ret;
   }
 
+  /** Finds the index of an expression in a list, comparing using
+   * {@link SqlNode#equalsDeep(SqlNode, Litmus)}. */
+  public static int indexOfDeep(List<? extends SqlNode> list, SqlNode e,
+      Litmus litmus) {
+    for (int i = 0; i < list.size(); i++) {
+      if (e.equalsDeep(list.get(i), litmus)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
   /**
    * Returns whether a node represents the NULL value.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibrary.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibrary.java
index d69e3985cf..930d96c506 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibrary.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibrary.java
@@ -50,6 +50,8 @@ public enum SqlLibrary {
   /** A collection of operators that are in Google BigQuery but not in standard
    * SQL. */
   BIG_QUERY("b", "bigquery"),
+  /** Calcite-specific extensions. */
+  CALCITE("c", "calcite"),
   /** A collection of operators that are in Apache Hive but not in standard
    * SQL. */
   HIVE("h", "hive"),
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index 1cde5d140f..85207325f2 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -51,6 +51,7 @@ import java.util.List;
 import java.util.Set;
 
 import static org.apache.calcite.sql.fun.SqlLibrary.BIG_QUERY;
+import static org.apache.calcite.sql.fun.SqlLibrary.CALCITE;
 import static org.apache.calcite.sql.fun.SqlLibrary.HIVE;
 import static org.apache.calcite.sql.fun.SqlLibrary.MSSQL;
 import static org.apache.calcite.sql.fun.SqlLibrary.MYSQL;
@@ -70,6 +71,17 @@ public abstract class SqlLibraryOperators {
   private SqlLibraryOperators() {
   }
 
+  /** The "AGGREGATE(m)" aggregate function;
+   * aggregates a measure column according to the measure's rollup strategy.
+   * This is a Calcite-specific extension.
+   *
+   * <p>This operator is for SQL (and AST); for internal use (RexNode and
+   * Aggregate) use {@code AGG_M2M}. */
+  @LibraryOperator(libraries = {CALCITE})
+  public static final SqlFunction AGGREGATE =
+      SqlBasicAggFunction.create("AGGREGATE", SqlKind.AGGREGATE_FN,
+          ReturnTypes.ARG0, OperandTypes.MEASURE);
+
   /** The "CONVERT_TIMEZONE(tz1, tz2, datetime)" function;
    * converts the timezone of {@code datetime} from {@code tz1} to {@code tz2}.
    * This function is only on Redshift, but we list it in PostgreSQL
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ApplySqlType.java b/core/src/main/java/org/apache/calcite/sql/type/ApplySqlType.java
new file mode 100644
index 0000000000..baef6c4ca2
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/type/ApplySqlType.java
@@ -0,0 +1,56 @@
+/*
+ * 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.calcite.sql.type;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.rel.type.RelDataType;
+
+import com.google.common.collect.ImmutableList;
+
+/** Type that applies generic type to type parameters. */
+abstract class ApplySqlType extends AbstractSqlType {
+  protected final ImmutableList<? extends RelDataType> types;
+
+  ApplySqlType(SqlTypeName typeName, boolean isNullable,
+      Iterable<? extends RelDataType> types) {
+    super(typeName, isNullable, null);
+    this.types = ImmutableList.copyOf(types);
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>Generate, for example, {@code MEASURE<INTEGER>}.
+   */
+  @Override protected void generateTypeString(StringBuilder sb, boolean withDetail) {
+    sb.append(typeName)
+        .append('<');
+    Ord.forEach(types, (type, i) -> {
+      if (i > 0) {
+        sb.append(", ");
+      }
+      if (withDetail) {
+        sb.append(type.getFullTypeString());
+      } else {
+        sb.append(type.toString());
+      }
+    });
+    sb.append('>');
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/type/MeasureSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/MeasureSqlType.java
new file mode 100644
index 0000000000..e6e4fb567c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/type/MeasureSqlType.java
@@ -0,0 +1,40 @@
+/*
+ * 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.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * Measure SQL type.
+ *
+ * <p>The type serves as a tag that the measure must be expanded
+ * into an expression before use.
+ */
+public class MeasureSqlType extends ApplySqlType {
+  /** Private constructor. */
+  private MeasureSqlType(RelDataType elementType, boolean isNullable) {
+    super(SqlTypeName.MEASURE, isNullable, ImmutableList.of(elementType));
+    computeDigest();
+  }
+
+  /** Creates a MeasureSqlType. */
+  static MeasureSqlType create(RelDataType elementType) {
+    return new MeasureSqlType(elementType, elementType.isNullable());
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index d0ffc19e5e..71ceef5815 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperandCountRange;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
@@ -658,6 +659,31 @@ public abstract class OperandTypes {
   public static final SqlSingleOperandTypeChecker CURSOR =
       family(SqlTypeFamily.CURSOR);
 
+  public static final SqlOperandTypeChecker MEASURE =
+      new FamilyOperandTypeChecker(ImmutableList.of(SqlTypeFamily.ANY),
+          i -> false) {
+        @Override public boolean checkSingleOperandType(
+            SqlCallBinding callBinding, SqlNode node,
+            int iFormalOperand, boolean throwOnFailure) {
+          if (!super.checkSingleOperandType(callBinding, node, iFormalOperand,
+              throwOnFailure)) {
+            return false;
+          }
+          // Scope is non-null at validate time, which is when we need to make
+          // this check.
+          final @Nullable SqlValidatorScope scope = callBinding.getScope();
+          if (scope != null && !scope.isMeasureRef(node)) {
+            if (throwOnFailure) {
+              throw callBinding.newValidationError(
+                  RESOURCE.argumentMustBeMeasure(
+                      callBinding.getOperator().getName()));
+            }
+            return false;
+          }
+          return true;
+        }
+      };
+
   /**
    * Parameter type-checking strategy where type must a nullable time interval,
    * nullable time interval.
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
index 8521c8419b..5dda51a331 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
@@ -119,6 +119,11 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
     return canonize(newType);
   }
 
+  @Override public RelDataType createMeasureType(RelDataType valueType) {
+    MeasureSqlType newType = MeasureSqlType.create(valueType);
+    return canonize(newType);
+  }
+
   @Override public RelDataType createSqlIntervalType(
       SqlIntervalQualifier intervalQualifier) {
     RelDataType newType =
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index 1392442d3a..4849fa1e9f 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -126,6 +126,7 @@ public enum SqlTypeName {
   /** Spatial type. Though not standard, it is common to several DBs, so we
    * do not flag it 'special' (internal). */
   GEOMETRY(PrecScale.NO_NO, false, ExtraSqlTypes.GEOMETRY, SqlTypeFamily.GEO),
+  MEASURE(PrecScale.NO_NO, true, Types.OTHER, SqlTypeFamily.ANY),
   SARG(PrecScale.NO_NO, true, Types.OTHER, SqlTypeFamily.ANY);
 
   public static final int MAX_DATETIME_PRECISION = 3;
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
index 66156157d4..92ce99f085 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeTransforms.java
@@ -195,6 +195,18 @@ public abstract class SqlTypeTransforms {
       (opBinding, typeToTransform) ->
           opBinding.getTypeFactory().createArrayType(typeToTransform, -1);
 
+  /** Parameter type-inference transform that transforms {@code T} to
+   * {@code MEASURE<T>} for some type T. */
+  public static final SqlTypeTransform TO_MEASURE =
+      (opBinding, typeToTransform) ->
+          opBinding.getTypeFactory().createMeasureType(typeToTransform);
+
+  /** Parameter type-inference transform that transforms {@code MEASURE<T>} to
+   * {@code T} for some type T. Inverse of {@link #TO_MEASURE}. */
+  public static final SqlTypeTransform FROM_MEASURE =
+      (opBinding, typeToTransform) ->
+          ((MeasureSqlType) typeToTransform).types.get(0);
+
   /**
    * Parameter type-inference transform strategy that wraps a given type in an array or
    * wraps a field of the given type in an array if the given type is struct with one field.
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java b/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
index c91bebac8d..0bcd91e937 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlVisitor.java
@@ -98,4 +98,9 @@ public interface SqlVisitor<R> {
    * @see SqlIntervalQualifier#accept(SqlVisitor)
    */
   R visit(SqlIntervalQualifier intervalQualifier);
+
+  /** Asks a {@code SqlNode} to accept this visitor. */
+  default R visitNode(SqlNode n) {
+    return n.accept(this);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
index fa04f8c031..170411eafe 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
@@ -27,6 +27,8 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.util.SqlBasicVisitor;
 import org.apache.calcite.util.Litmus;
 
+import com.google.common.collect.Iterables;
+
 import java.util.ArrayDeque;
 import java.util.Deque;
 import java.util.List;
@@ -45,9 +47,10 @@ class AggChecker extends SqlBasicVisitor<Void> {
 
   private final Deque<SqlValidatorScope> scopes = new ArrayDeque<>();
   private final List<SqlNode> extraExprs;
+  private final List<SqlNode> measureExprs;
   private final List<SqlNode> groupExprs;
-  private boolean distinct;
-  private SqlValidatorImpl validator;
+  private final boolean distinct;
+  private final SqlValidatorImpl validator;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -56,6 +59,9 @@ class AggChecker extends SqlBasicVisitor<Void> {
    *
    * @param validator  Validator
    * @param scope      Scope
+   * @param extraExprs Expressions in GROUP BY (or SELECT DISTINCT) clause,
+   *                   that are therefore available
+   * @param measureExprs Expressions that are the names of measures
    * @param groupExprs Expressions in GROUP BY (or SELECT DISTINCT) clause,
    *                   that are therefore available
    * @param distinct   Whether aggregation checking is because of a SELECT
@@ -65,10 +71,12 @@ class AggChecker extends SqlBasicVisitor<Void> {
       SqlValidatorImpl validator,
       AggregatingScope scope,
       List<SqlNode> extraExprs,
+      List<SqlNode> measureExprs,
       List<SqlNode> groupExprs,
       boolean distinct) {
     this.validator = validator;
     this.extraExprs = extraExprs;
+    this.measureExprs = measureExprs;
     this.groupExprs = groupExprs;
     this.distinct = distinct;
     this.scopes.push(scope);
@@ -76,15 +84,18 @@ class AggChecker extends SqlBasicVisitor<Void> {
 
   //~ Methods ----------------------------------------------------------------
 
-  boolean isGroupExpr(SqlNode expr) {
-    for (SqlNode groupExpr : groupExprs) {
-      if (groupExpr.equalsDeep(expr, Litmus.IGNORE)) {
+  boolean isGroupExpr(SqlNode e) {
+    for (SqlNode expr : Iterables.concat(extraExprs, measureExprs, groupExprs)) {
+      if (expr.equalsDeep(e, Litmus.IGNORE)) {
         return true;
       }
     }
+    return false;
+  }
 
-    for (SqlNode extraExpr : extraExprs) {
-      if (extraExpr.equalsDeep(expr, Litmus.IGNORE)) {
+  boolean isMeasureExp(SqlNode e) {
+    for (SqlNode expr : measureExprs) {
+      if (expr.equalsDeep(e, Litmus.IGNORE)) {
         return true;
       }
     }
@@ -92,11 +103,22 @@ class AggChecker extends SqlBasicVisitor<Void> {
   }
 
   @Override public Void visit(SqlIdentifier id) {
-    if (isGroupExpr(id) || id.isStar()) {
+    if (id.isStar()) {
       // Star may validly occur in "SELECT COUNT(*) OVER w"
       return null;
     }
 
+    if (!validator.config().nakedMeasures()
+        && isMeasureExp(id)) {
+      SqlNode originalExpr = validator.getOriginal(id);
+      throw validator.newValidationError(originalExpr,
+          RESOURCE.measureIllegal());
+    }
+
+    if (isGroupExpr(id)) {
+      return null;
+    }
+
     // Is it a call to a parentheses-free function?
     final SqlCall call = validator.makeNullaryCall(id);
     if (call != null) {
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
index 9c11afa6a5..ea36731b2c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
@@ -24,9 +24,9 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
-import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -127,12 +127,18 @@ public class AggregatingSelectScope
    * DISTINCT clause, if distinct) and that can therefore be referenced
    * without being wrapped in aggregate functions.
    *
+   * <p>Also identifies measure expressions, which are not in {@code GROUP BY}
+   * but can still be referenced without aggregate functions. (Some dialects
+   * require measures to be wrapped in
+   * {@link org.apache.calcite.sql.fun.SqlLibraryOperators#AGGREGATE};
+   * see {@link SqlValidator.Config#nakedMeasures()}.)
+   *
    * <p>The expressions are fully-qualified, and any "*" in select clauses are
    * expanded.
-   *
-   * @return list of grouping expressions
    */
-  private Pair<ImmutableList<SqlNode>, ImmutableList<SqlNode>> getGroupExprs() {
+  private void gatherGroupExprs(ImmutableList.Builder<SqlNode> extraExprs,
+      ImmutableList.Builder<SqlNode> measureExprs,
+      ImmutableList.Builder<SqlNode> groupExprs) {
     if (distinct) {
       // Cannot compute this in the constructor: select list has not been
       // expanded yet.
@@ -140,7 +146,6 @@ public class AggregatingSelectScope
 
       // Remove the AS operator so the expressions are consistent with
       // OrderExpressionExpander.
-      ImmutableList.Builder<SqlNode> groupExprs = ImmutableList.builder();
       final SelectScope selectScope = (SelectScope) parent;
       List<SqlNode> expandedSelectList = Objects.requireNonNull(
           selectScope.getExpandedSelectList(),
@@ -148,19 +153,19 @@ public class AggregatingSelectScope
       for (SqlNode selectItem : expandedSelectList) {
         groupExprs.add(stripAs(selectItem));
       }
-      return Pair.of(ImmutableList.of(), groupExprs.build());
-    } else if (select.getGroup() != null) {
+    } else {
       SqlValidatorUtil.GroupAnalyzer groupAnalyzer = this.groupAnalyzer;
       if (groupAnalyzer != null) {
         // we are in the middle of resolving
-        return Pair.of(ImmutableList.of(),
-            ImmutableList.copyOf(groupAnalyzer.groupExprs));
+        extraExprs.addAll(groupAnalyzer.extraExprs);
+        measureExprs.addAll(groupAnalyzer.measureExprs);
+        groupExprs.addAll(groupAnalyzer.groupExprs);
       } else {
         final Resolved resolved = this.resolved.get();
-        return Pair.of(resolved.extraExprList, resolved.groupExprList);
+        extraExprs.addAll(resolved.extraExprList);
+        measureExprs.addAll(resolved.measureExprList);
+        groupExprs.addAll(resolved.groupExprList);
       }
-    } else {
-      return Pair.of(ImmutableList.of(), ImmutableList.of());
     }
   }
 
@@ -215,9 +220,13 @@ public class AggregatingSelectScope
     }
 
     // Make sure expression is valid, throws if not.
-    Pair<ImmutableList<SqlNode>, ImmutableList<SqlNode>> pair = getGroupExprs();
+    final ImmutableList.Builder<SqlNode> extraExprs = ImmutableList.builder();
+    final ImmutableList.Builder<SqlNode> measureExprs = ImmutableList.builder();
+    final ImmutableList.Builder<SqlNode> groupExprs = ImmutableList.builder();
+    gatherGroupExprs(extraExprs, measureExprs, groupExprs);
     final AggChecker aggChecker =
-        new AggChecker(validator, this, pair.left, pair.right, distinct);
+        new AggChecker(validator, this, extraExprs.build(),
+            measureExprs.build(), groupExprs.build(), distinct);
     if (deep) {
       expr.accept(aggChecker);
     }
@@ -236,15 +245,17 @@ public class AggregatingSelectScope
    * the scope is created. */
   public static class Resolved {
     public final ImmutableList<SqlNode> extraExprList;
+    public final ImmutableList<SqlNode> measureExprList;
     public final ImmutableList<SqlNode> groupExprList;
     public final ImmutableBitSet groupSet;
     public final ImmutableSortedMultiset<ImmutableBitSet> groupSets;
     public final Map<Integer, Integer> groupExprProjection;
 
-    Resolved(List<SqlNode> extraExprList, List<SqlNode> groupExprList,
-        Iterable<ImmutableBitSet> groupSets,
+    Resolved(List<SqlNode> extraExprList, List<SqlNode> measureExprList,
+        List<SqlNode> groupExprList, Iterable<ImmutableBitSet> groupSets,
         Map<Integer, Integer> groupExprProjection) {
       this.extraExprList = ImmutableList.copyOf(extraExprList);
+      this.measureExprList = ImmutableList.copyOf(measureExprList);
       this.groupExprList = ImmutableList.copyOf(groupExprList);
       this.groupSet = ImmutableBitSet.range(groupExprList.size());
       this.groupSets = ImmutableSortedMultiset.copyOf(groupSets);
@@ -263,12 +274,7 @@ public class AggregatingSelectScope
     }
 
     public int lookupGroupingExpr(SqlNode operand) {
-      for (Ord<SqlNode> groupExpr : Ord.zip(groupExprList)) {
-        if (operand.equalsDeep(groupExpr.e, Litmus.IGNORE)) {
-          return groupExpr.i;
-        }
-      }
-      return -1;
+      return SqlUtil.indexOfDeep(groupExprList, operand, Litmus.IGNORE);
     }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
index 23d82169c2..0c4e7b11e9 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
@@ -30,6 +30,7 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlWindow;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -40,6 +41,7 @@ import com.google.common.collect.ImmutableList;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -618,13 +620,28 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     if (selectScope != null) {
       // Find all expressions in this scope that reference measures
       for (ScopeChild child : selectScope.children) {
+        final RelDataType rowType = child.namespace.getRowType();
         if (child.namespace instanceof SelectNamespace) {
           final SqlSelect select = ((SelectNamespace) child.namespace).getNode();
           Pair.forEach(select.getSelectList(),
-              child.namespace.getRowType().getFieldList(),
+              rowType.getFieldList(),
               (selectItem, field) -> {
-                // TODO
+                if (SqlValidatorUtil.isMeasure(selectItem)) {
+                  analyzer.measureExprs.add(
+                      new SqlIdentifier(
+                          Arrays.asList(child.name, field.getName()),
+                          SqlParserPos.ZERO));
+                }
               });
+        } else {
+          rowType.getFieldList().forEach(field -> {
+            if (field.getType().getSqlTypeName() == SqlTypeName.MEASURE) {
+              analyzer.measureExprs.add(
+                  new SqlIdentifier(
+                      Arrays.asList(child.name, field.getName()),
+                      SqlParserPos.ZERO));
+            }
+          });
         }
       }
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index c052411870..50e2151f1e 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -889,6 +889,16 @@ public interface SqlValidator {
      */
     Config withLenientOperatorLookup(boolean lenient);
 
+    /** Returns whether the validator allows measures to be used without the
+     * AGGREGATE function. Default is true. */
+    @Value.Default default boolean nakedMeasures() {
+      return true;
+    }
+
+    /** Sets whether the validator allows measures to be used without the
+     * AGGREGATE function. */
+    Config withNakedMeasures(boolean nakedMeasures);
+
     /** Returns whether the validator supports implicit type coercion. */
     @Value.Default default boolean typeCoercionEnabled() {
       return true;
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index b42c88582b..0db4eaa8a4 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -4593,6 +4593,16 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       }
     }
 
+    // Unless 'naked measures' are enabled, a non-aggregating query cannot
+    // reference measure columns. (An aggregating query can use them as
+    // argument to the AGGREGATE function.)
+    if (!config.nakedMeasures()
+        && !(scope instanceof AggregatingScope)
+        && scope.isMeasureRef(expr)) {
+      throw newValidationError(expr,
+          RESOURCE.measureMustBeInAggregateQuery());
+    }
+
     // Call on the expression to validate itself.
     expr.validateExpr(this, scope);
 
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
index 83b98acdc2..92b6b16dcf 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.StructKind;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
@@ -24,6 +25,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -123,6 +125,26 @@ public interface SqlValidatorScope {
    */
   SqlQualified fullyQualify(SqlIdentifier identifier);
 
+  /** Returns whether an expression is a reference to a measure column. */
+  default boolean isMeasureRef(SqlNode node) {
+    if (node instanceof SqlIdentifier) {
+      final SqlQualified q = fullyQualify((SqlIdentifier) node);
+      if (q.suffix().size() == 1
+          && q.namespace != null) {
+        final @Nullable RelDataTypeField f =
+            q.namespace.field(q.suffix().get(0));
+        if (q.namespace instanceof SelectNamespace) {
+          final SqlSelect select = ((SelectNamespace) q.namespace).getNode();
+          return f != null
+              && SqlValidatorUtil.isMeasure(select.getSelectList().get(f.getIndex()));
+        }
+        return f != null
+            && f.getType().getSqlTypeName() == SqlTypeName.MEASURE;
+      }
+    }
+    return false;
+  }
+
   /**
    * Registers a relation in this scope.
    *
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index 6cd92286ba..74e6c9ec59 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -1287,6 +1287,22 @@ public class SqlValidatorUtil {
     }
   }
 
+  /** Returns whether a select item is a measure. */
+  public static boolean isMeasure(SqlNode selectItem) {
+    return getMeasure(selectItem) != null;
+  }
+
+  /** Returns the measure expression if a select item is a measure, null
+   * otherwise.
+   *
+   * <p>For a measure, {@code selectItem} will have the form
+   * {@code AS(MEASURE(exp), alias)} and this method returns {@code exp}. */
+  public static @Nullable SqlNode getMeasure(SqlNode selectItem) {
+    // The implementation of this method will be extended when we add the
+    // 'AS MEASURE' construct in [CALCITE-4496].
+    return null;
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**
@@ -1373,12 +1389,13 @@ public class SqlValidatorUtil {
     /** Extra expressions, computed from the input as extra GROUP BY
      * expressions. For example, calls to the {@code TUMBLE} functions. */
     final List<SqlNode> extraExprs = new ArrayList<>();
+    final List<SqlNode> measureExprs = new ArrayList<>();
     final List<SqlNode> groupExprs = new ArrayList<>();
     final Map<Integer, Integer> groupExprProjection = new HashMap<>();
     final List<ImmutableBitSet> flatGroupSets = new ArrayList<>();
 
     AggregatingSelectScope.Resolved finish() {
-      return new AggregatingSelectScope.Resolved(extraExprs,
+      return new AggregatingSelectScope.Resolved(extraExprs, measureExprs,
           groupExprs, flatGroupSets, groupExprProjection);
     }
   }
diff --git a/core/src/main/java/org/apache/calcite/util/Litmus.java b/core/src/main/java/org/apache/calcite/util/Litmus.java
index dd62f50e47..863e14964b 100644
--- a/core/src/main/java/org/apache/calcite/util/Litmus.java
+++ b/core/src/main/java/org/apache/calcite/util/Litmus.java
@@ -25,25 +25,10 @@ import org.slf4j.helpers.MessageFormatter;
 public interface Litmus {
   /** Implementation of {@link org.apache.calcite.util.Litmus} that throws
    * an {@link java.lang.AssertionError} on failure. */
-  Litmus THROW = new Litmus() {
-    @Override public boolean fail(@Nullable String message, @Nullable Object... args) {
-      final String s = message == null
-          ? null : MessageFormatter.arrayFormat(message, args).getMessage();
-      throw new AssertionError(s);
-    }
-
-    @Override public boolean succeed() {
-      return true;
-    }
-
-    @Override public boolean check(boolean condition, @Nullable String message,
-        @Nullable Object... args) {
-      if (condition) {
-        return succeed();
-      } else {
-        return fail(message, args);
-      }
-    }
+  Litmus THROW = (message, args) -> {
+    final String s = message == null
+        ? null : MessageFormatter.arrayFormat(message, args).getMessage();
+    throw new AssertionError(s);
   };
 
   /** Implementation of {@link org.apache.calcite.util.Litmus} that returns
@@ -53,14 +38,16 @@ public interface Litmus {
       return false;
     }
 
-    @Override public boolean succeed() {
-      return true;
-    }
-
     @Override public boolean check(boolean condition, @Nullable String message,
         @Nullable Object... args) {
       return condition;
     }
+
+    @Override public Litmus withMessageArgs(@Nullable String message,
+        @Nullable Object... args) {
+      // IGNORE never throws, so don't bother remembering message and args.
+      return this;
+    }
   };
 
   /** Called when test fails. Returns false or throws.
@@ -71,7 +58,9 @@ public interface Litmus {
   boolean fail(@Nullable String message, @Nullable Object... args);
 
   /** Called when test succeeds. Returns true. */
-  boolean succeed();
+  default boolean succeed() {
+    return true;
+  }
 
   /** Checks a condition.
    *
@@ -79,5 +68,19 @@ public interface Litmus {
    * if the condition is false, calls {@link #fail},
    * converting {@code info} into a string message.
    */
-  boolean check(boolean condition, @Nullable String message, @Nullable Object... args);
+  default boolean check(boolean condition, @Nullable String message,
+      @Nullable Object... args) {
+    if (condition) {
+      return succeed();
+    } else {
+      return fail(message, args);
+    }
+  }
+
+  /** Creates a Litmus that, if it fails, will use the given arguments. */
+  default Litmus withMessageArgs(@Nullable String message,
+      @Nullable Object... args) {
+    final Litmus delegate = this;
+    return (message1, args1) -> delegate.fail(message, args);
+  }
 }
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index c904535c98..cd8135ff43 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -109,6 +109,7 @@ CrossJoinDisallowsCondition=Cannot specify condition (NATURAL keyword, or ON or
 NaturalDisallowsOnOrUsing=Cannot specify NATURAL keyword with ON or USING clause
 ColumnInUsingNotUnique=Column name ''{0}'' in NATURAL join or USING clause is not unique on one side of join
 NaturalOrUsingColumnNotCompatible=Column ''{0}'' matched using NATURAL keyword or USING clause has incompatible types: cannot compare ''{1}'' to ''{2}''
+ArgumentMustBeMeasure=Argument to function ''{0}'' must be a measure
 WindowNotFound=Window ''{0}'' not found
 DisallowsNullTreatment=Cannot specify IGNORE NULLS or RESPECT NULLS following ''{0}''
 NotGroupExpr=Expression ''{0}'' is not being grouped
@@ -119,6 +120,8 @@ NotSelectDistinctExpr=Expression ''{0}'' is not in the select clause
 AggregateIllegalInClause=Aggregate expression is illegal in {0} clause
 WindowedAggregateIllegalInClause=Windowed aggregate expression is illegal in {0} clause
 NestedAggIllegal=Aggregate expressions cannot be nested
+MeasureIllegal=Measure expressions can only occur within AGGREGATE function
+MeasureMustBeInAggregateQuery=Measure expressions can only occur within a GROUP BY query
 AggregateInFilterIllegal=FILTER must not contain aggregate expression
 AggregateInWithinGroupIllegal=WITHIN GROUP must not contain aggregate expression
 AggregateInWithinDistinctIllegal=WITHIN DISTINCT must not contain aggregate expression
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index b306703a0c..ad3c6cc0a5 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -40,10 +40,12 @@ import org.apache.calcite.rel.logical.LogicalTableModify;
 import org.apache.calcite.rel.rules.CoreRules;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.fun.SqlLibrary;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlDelegatingConformance;
+import org.apache.calcite.test.catalog.MockCatalogReaderExtended;
 import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
@@ -4410,6 +4412,22 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  /** Tests a reference to a measure column in an underlying table. The measure
+   * is treated as a black box: it is not expanded, just wrapped in a call to
+   * AGGREGATE. */
+  @Test void testMeasureRef() {
+    final String sql = "select deptno, aggregate(count_plus_100) as c\n"
+        + "from empm\n"
+        + "group by deptno";
+    fixture()
+        .withFactory(c ->
+            c.withOperatorTable(t ->
+                SqlValidatorTest.operatorTableFor(SqlLibrary.CALCITE)))
+        .withCatalogReader(MockCatalogReaderExtended::create)
+        .withSql(sql)
+        .ok();
+  }
+
   /** Test case for:
    * <a href="https://issues.apache.org/jira/browse/CALCITE-3310">[CALCITE-3310]
    * Approximate and exact aggregate calls are recognized as the same
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index d972c830bb..b25fd2b666 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -5372,6 +5372,96 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .type("RecordType(CHAR(2) NOT NULL A, INTEGER NOT NULL B) NOT NULL");
   }
 
+  @Test void testMeasureRef() {
+    // A measure can be used in the SELECT clause of a GROUP BY query even
+    // though it is not a GROUP BY key.
+    SqlValidatorFixture f =
+        fixture().withExtendedCatalog()
+            .withOperatorTable(operatorTableFor(SqlLibrary.CALCITE));
+    SqlValidatorFixture f2 =
+        f.withValidatorConfig(c -> c.withNakedMeasures(false));
+
+    final String measureIllegal =
+        "Measure expressions can only occur within AGGREGATE function";
+    final String measureIllegal2 =
+        "Measure expressions can only occur within a GROUP BY query";
+
+    final String sql0 = "select deptno, ^count_plus_100^\n"
+        + "from empm\n"
+        + "group by deptno";
+    f.withSql(sql0)
+        .isAggregate(is(true))
+        .ok();
+
+    // Same SQL is invalid if naked measures are not enabled
+    f2.withSql(sql0).fails(measureIllegal);
+
+    // Similarly, with alias
+    final String sql1b = "select deptno, ^count_plus_100^ as x\n"
+        + "from empm\n"
+        + "group by deptno";
+    f.withSql(sql1b).isAggregate(is(true)).ok();
+    f2.withSql(sql1b).fails(measureIllegal);
+
+    // Similarly, in an expression
+    final String sql1c = "select deptno, deptno + ^count_plus_100^ * 2 as x\n"
+        + "from empm\n"
+        + "group by deptno";
+    f.withSql(sql1c).isAggregate(is(true)).ok();
+    f2.withSql(sql1c).fails(measureIllegal);
+
+    // Similarly, for a query that is an aggregate query because of another
+    // aggregate function.
+    final String sql1 = "select count(*), ^count_plus_100^\n"
+        + "from empm";
+    f.withSql(sql1).isAggregate(is(true)).ok();
+    f2.withSql(sql1).fails(measureIllegal);
+
+    // A measure in a non-aggregate query.
+    // Using a measure should not make it an aggregate query.
+    // The type of the measure should be the result type of the COUNT aggregate
+    // function (BIGINT), not type of the un-aggregated argument type (VARCHAR).
+    final String sql2 = "select deptno, ^count_plus_100^, ename\n"
+        + "from empm";
+    f.withSql(sql2)
+        .type("RecordType(INTEGER NOT NULL DEPTNO, "
+            + "MEASURE<INTEGER NOT NULL> NOT NULL COUNT_PLUS_100, "
+            + "VARCHAR(20) NOT NULL ENAME) NOT NULL")
+        .isAggregate(is(false));
+    f2.withSql(sql2).fails(measureIllegal2);
+
+    // as above, wrapping the measure in AGGREGATE
+    final String sql3 = "select deptno, aggregate(count_plus_100) as x, ename\n"
+        + "from empm\n"
+        + "group by deptno, ename";
+    f.withSql(sql3)
+        .type("RecordType(INTEGER NOT NULL DEPTNO, "
+            + "MEASURE<INTEGER NOT NULL> NOT NULL X, "
+            + "VARCHAR(20) NOT NULL ENAME) NOT NULL");
+
+    // you can apply the AGGREGATE function only to measures
+    f.withSql("select deptno, aggregate(count_plus_100), ^aggregate(ename)^\n"
+            + "from empm\n"
+            + "group by deptno, ename")
+        .fails("Argument to function 'AGGREGATE' must be a measure");
+
+    f.withSql("select deptno, ^aggregate(count_plus_100 + 1)^\n"
+            + "from empm\n"
+            + "group by deptno, ename")
+        .fails("Argument to function 'AGGREGATE' must be a measure");
+
+    // A query with AGGREGATE is an aggregate query, even without GROUP BY,
+    // and even if it is inside an expression.
+    f.withSql("select aggregate(count_plus_100) + 1\n"
+            + "from empm")
+        .isAggregate(is(true));
+
+    // Including a measure in a query does not make it an aggregate query
+    f.withSql("select count_plus_100\n"
+            + "from empm")
+        .isAggregate(is(false));
+  }
+
   @Test void testAmbiguousColumnInIn() {
     // ok: cyclic reference
     sql("select * from emp as e\n"
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index e492b86280..bcb6066040 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -4338,6 +4338,20 @@ LogicalProject(START_MGR=[$0], BOTTOM_MGR=[$1], END_MGR=[$2])
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
   LogicalMatch(partition=[[]], order=[[]], outputFields=[[EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO, SLACKER]], allRows=[false], after=[SKIP TO LAST('DOWN')], pattern=[(('STRT', PATTERN_QUANTIFIER('DOWN', 1, -1, false)), PATTERN_QUANTIFIER('UP', 1, -1, false))], isStrictStarts=[false], isStrictEnds=[false], subsets=[[[DOWN, STRT]]], patternDefinitions=[[<(PREV(DOWN.$3, 0), PREV(DOWN.$3, 1)), >(PREV(UP.$3, 0), NEXT(PREV(UP.$3, 0), 1))]], inputFields=[[EMPNO, ENAME, JOB, MGR, HIR [...]
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMeasureRef">
+    <Resource name="sql">
+      <![CDATA[select deptno, aggregate(count_plus_100) as c
+from empm
+group by deptno]]>
+    </Resource>
+    <Resource name="plan">
+      <![CDATA[
+LogicalAggregate(group=[{0}], C=[AGGREGATE($1)])
+  LogicalProject(DEPTNO=[$7], COUNT_PLUS_100=[$9])
+    LogicalTableScan(table=[[CATALOG, SALES, EMPM]])
 ]]>
     </Resource>
   </TestCase>
diff --git a/core/src/test/resources/sql/measure.iq b/core/src/test/resources/sql/measure.iq
new file mode 100644
index 0000000000..6ba46a76ca
--- /dev/null
+++ b/core/src/test/resources/sql/measure.iq
@@ -0,0 +1,44 @@
+# measure.iq - measure expressions
+#
+# 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.
+#
+!set outputformat mysql
+!use scott
+
+!if (false) {
+
+# Measure that references another measure
+select *
+from (
+  select deptno,
+    empno + 1 as measure e1,
+    e1 + deptno as measure e2
+  from "scott".emp
+  where deptno = 10);
++--------+------+------+
+| DEPTNO | E1   | E2   |
++--------+------+------+
+|     10 | 7783 | 7793 |
+|     10 | 7840 | 7850 |
+|     10 | 7935 | 7945 |
++--------+------+------+
+(3 rows)
+
+!ok
+
+!}
+
+# End measure.iq
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index ddf4698ccb..fe11d27e3c 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -49,6 +49,12 @@ here to appease testAllFunctionsAreDocumented:
 | SUCCEEDS       | Documented as a period operator
 | TABLE          | Documented as part of FROM syntax
 | VARIANCE()     | In SqlStdOperatorTable, but not fully implemented
+
+Dialect-specific:
+
+| C | Function       | Reason not documented
+|:--|:-------------- |:---------------------
+
 {% endcomment %}
 -->
 
@@ -2563,6 +2569,7 @@ connect string parameter.
 
 The 'C' (compatibility) column contains value:
 * 'b' for Google BigQuery ('fun=bigquery' in the connect string),
+* 'c' for Apache Calcite ('fun=calcite' in the connect string),
 * 'h' for Apache Hive ('fun=hive' in the connect string),
 * 'm' for MySQL ('fun=mysql' in the connect string),
 * 'q' for Microsoft SQL Server ('fun=mssql' in the connect string),
@@ -2679,6 +2686,7 @@ Dialect-specific aggregate functions.
 
 | C | Operator syntax                                | Description
 |:- |:-----------------------------------------------|:-----------
+| c | AGGREGATE(m)                                   | Computes measure *m* in the context of the current GROUP BY key
 | b p | ARRAY_AGG( [ ALL &#124; DISTINCT ] value [ RESPECT NULLS &#124; IGNORE NULLS ] [ ORDER BY orderItem [, orderItem ]* ] ) | Gathers values into arrays
 | b p | ARRAY_CONCAT_AGG( [ ALL &#124; DISTINCT ] value [ ORDER BY orderItem [, orderItem ]* ] ) | Concatenates arrays into arrays
 | p | BOOL_AND(condition)                            | Synonym for `EVERY`
diff --git a/testkit/src/main/java/org/apache/calcite/test/SqlToRelTestBase.java b/testkit/src/main/java/org/apache/calcite/test/SqlToRelTestBase.java
index 4b86d4656d..05d5b4efdd 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -60,7 +60,7 @@ public abstract class SqlToRelTestBase {
    * Default catalog: {@link org.apache.calcite.test.catalog.MockCatalogReaderSimple#init()}.
    */
   public final SqlToRelFixture sql(String sql) {
-    return fixture().expression(false).withSql(sql);
+    return fixture().withSql(sql);
   }
 
   public final SqlToRelFixture expr(String sql) {
diff --git a/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java b/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
index c9890a5915..2ebc97b47a 100644
--- a/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
+++ b/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReaderExtended.java
@@ -68,6 +68,7 @@ public class MockCatalogReaderExtended extends MockCatalogReaderSimple {
   @Override public MockCatalogReaderExtended init() {
     super.init();
 
+    final Fixture f = new Fixture(typeFactory);
     MockSchema salesSchema = new MockSchema("SALES");
     // Same as "EMP_20" except it uses ModifiableViewTable which populates
     // constrained columns with default values on INSERT and has a single constraint on DEPTNO.
@@ -114,9 +115,25 @@ public class MockCatalogReaderExtended extends MockCatalogReaderSimple {
         empModifiableViewNames3.get(2), false, 20, null);
     registerTable(mockEmpViewTable3);
 
+    // Register "EMPM" table.
+    // Same as "EMP" but with a "COUNT_PLUS_100" measure column.
+    final MockTable empmTable =
+        MockTable.create(this, salesSchema, "EMPM", false, 14);
+    empmTable.addColumn("EMPNO", f.intType, true);
+    empmTable.addColumn("ENAME", f.varchar20Type);
+    empmTable.addColumn("JOB", f.varchar10Type);
+    empmTable.addColumn("MGR", f.intTypeNull);
+    empmTable.addColumn("HIREDATE", f.timestampType);
+    empmTable.addColumn("SAL", f.intType);
+    empmTable.addColumn("COMM", f.intType);
+    empmTable.addColumn("DEPTNO", f.intType);
+    empmTable.addColumn("SLACKER", f.booleanType);
+    empmTable.addColumn("COUNT_PLUS_100",
+        f.typeFactory.createMeasureType(f.intType));
+    registerTable(empmTable);
+
     MockSchema structTypeSchema = new MockSchema("STRUCT");
     registerSchema(structTypeSchema);
-    final Fixture f = new Fixture(typeFactory);
     final List<CompoundNameColumn> columnsExtended = Arrays.asList(
         new CompoundNameColumn("", "K0", f.varchar20TypeNull),
         new CompoundNameColumn("", "C1", f.varchar20TypeNull),