You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/10/18 16:02:09 UTC

[GitHub] [druid] paul-rogers commented on a diff in pull request #13165: Druid Catalog basics

paul-rogers commented on code in PR #13165:
URL: https://github.com/apache/druid/pull/13165#discussion_r997643796


##########
processing/src/main/java/org/apache/druid/segment/column/RowSignature.java:
##########
@@ -32,6 +32,7 @@
 import org.apache.druid.segment.ColumnInspector;
 
 import javax.annotation.Nullable;
+

Review Comment:
   Sorry. Eclipse kinda likes to change these...



##########
server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.joda.time.Period;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class CatalogUtils
+{
+  // Amazing that a parser doesn't already exist...
+  private static final Map<String, Granularity> GRANULARITIES = new HashMap<>();
+
+  static {
+    GRANULARITIES.put("millisecond", Granularities.SECOND);

Review Comment:
   We could. This list was assembled from multiple places where define granularities. Now that we have the list, yes, we could simplify. It has been suggested to just allow, say, the ISO format. However native specs use the other formats, so I erred on the side of compatibility.
   
   OK, so I'm just going to delete the table and require ISO format. This saves having to deal with all the odd cases here and in the UI.



##########
server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.joda.time.Period;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class CatalogUtils
+{
+  // Amazing that a parser doesn't already exist...
+  private static final Map<String, Granularity> GRANULARITIES = new HashMap<>();
+
+  static {
+    GRANULARITIES.put("millisecond", Granularities.SECOND);
+    GRANULARITIES.put("second", Granularities.SECOND);
+    GRANULARITIES.put("minute", Granularities.MINUTE);
+    GRANULARITIES.put("5 minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("5 minutes", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("five_minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("10 minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("10 minutes", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("ten_minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("15 minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("15 minutes", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("fifteen_minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("30 minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("30 minutes", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("thirty_minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("hour", Granularities.HOUR);
+    GRANULARITIES.put("6 hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("6 hours", Granularities.SIX_HOUR);
+    GRANULARITIES.put("six_hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("day", Granularities.DAY);
+    GRANULARITIES.put("week", Granularities.WEEK);
+    GRANULARITIES.put("month", Granularities.MONTH);
+    GRANULARITIES.put("quarter", Granularities.QUARTER);
+    GRANULARITIES.put("year", Granularities.YEAR);
+    GRANULARITIES.put("all", Granularities.ALL);
+  }
+
+  public static Granularity toGranularity(String value)
+  {
+    return GRANULARITIES.get(StringUtils.toLowerCase(value));
+  }
+
+  public static int findColumn(List<ColumnSpec> columns, String colName)
+  {
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).name().equals(colName)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static List<String> columnNames(List<ColumnSpec> columns)
+  {
+    return columns
+           .stream()
+           .map(col -> col.name())
+           .collect(Collectors.toList());
+  }
+
+  public static <T extends ColumnSpec> List<T> dropColumns(
+      final List<T> columns,
+      final List<String> toDrop)
+  {
+    if (toDrop == null || toDrop.isEmpty()) {
+      return columns;
+    }
+    Set<String> drop = new HashSet<String>(toDrop);
+    List<T> revised = new ArrayList<>();
+    for (T col : columns) {

Review Comment:
   Again, could be, but this is a really simple loop.



##########
server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.joda.time.Period;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class CatalogUtils
+{
+  // Amazing that a parser doesn't already exist...
+  private static final Map<String, Granularity> GRANULARITIES = new HashMap<>();
+
+  static {
+    GRANULARITIES.put("millisecond", Granularities.SECOND);
+    GRANULARITIES.put("second", Granularities.SECOND);
+    GRANULARITIES.put("minute", Granularities.MINUTE);
+    GRANULARITIES.put("5 minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("5 minutes", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("five_minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("10 minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("10 minutes", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("ten_minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("15 minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("15 minutes", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("fifteen_minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("30 minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("30 minutes", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("thirty_minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("hour", Granularities.HOUR);
+    GRANULARITIES.put("6 hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("6 hours", Granularities.SIX_HOUR);
+    GRANULARITIES.put("six_hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("day", Granularities.DAY);
+    GRANULARITIES.put("week", Granularities.WEEK);
+    GRANULARITIES.put("month", Granularities.MONTH);
+    GRANULARITIES.put("quarter", Granularities.QUARTER);
+    GRANULARITIES.put("year", Granularities.YEAR);
+    GRANULARITIES.put("all", Granularities.ALL);
+  }
+
+  public static Granularity toGranularity(String value)
+  {
+    return GRANULARITIES.get(StringUtils.toLowerCase(value));
+  }
+
+  public static int findColumn(List<ColumnSpec> columns, String colName)
+  {
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).name().equals(colName)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static List<String> columnNames(List<ColumnSpec> columns)
+  {
+    return columns
+           .stream()
+           .map(col -> col.name())
+           .collect(Collectors.toList());
+  }
+
+  public static <T extends ColumnSpec> List<T> dropColumns(
+      final List<T> columns,
+      final List<String> toDrop)
+  {
+    if (toDrop == null || toDrop.isEmpty()) {
+      return columns;
+    }
+    Set<String> drop = new HashSet<String>(toDrop);
+    List<T> revised = new ArrayList<>();
+    for (T col : columns) {
+      if (!drop.contains(col.name())) {
+        revised.add(col);
+      }
+    }
+    return revised;
+  }
+
+  /**
+   * Convert a catalog granularity string to the Druid form. Catalog granularities
+   * are either the usual descriptive strings (in any case), or an ISO period.
+   * For the odd interval, the interval name is also accepted (for the other
+   * intervals, the interval name is the descriptive string).
+   */
+  public static Granularity asDruidGranularity(String value)
+  {
+    if (Strings.isNullOrEmpty(value)) {
+      return Granularities.ALL;
+    }
+    Granularity gran = toGranularity(value);
+    if (gran != null) {
+      return gran;
+    }
+
+    try {
+      return new PeriodGranularity(new Period(value), null, null);
+    }
+    catch (IllegalArgumentException e) {
+      throw new IAE(StringUtils.format("%s is an invalid period string", value));
+    }
+  }
+
+  /**
+   * {@code String}-to-{@code List<String>} conversion. The string can contain zero items,
+   * one items, or a list. The list items are separated by a comma and optional
+   * whitespace.
+   */
+  public static List<String> stringToList(String value)
+  {
+    if (value == null) {
+      return null;
+    }
+    return Arrays.asList(value.split(",\\s*"));
+  }
+
+  public static <T> T safeCast(Object value, Class<T> type, String propertyName)
+  {
+    if (value == null) {
+      return null;
+    }
+    try {
+      return type.cast(value);
+    }
+    catch (ClassCastException e) {
+      throw new IAE("Value [%s] is not valid for property %s, expected type %s",
+          value,
+          propertyName,
+          type.getSimpleName()
+      );
+    }
+  }
+
+  public static <T> T safeGet(Map<String, Object> map, String propertyName, Class<T> type)
+  {
+    return safeCast(map.get(propertyName), type, propertyName);
+  }
+
+  public static String stringListToLines(List<String> lines)
+  {
+    if (lines.isEmpty()) {
+      return "";
+    }
+    return String.join("\n", lines) + "\n";
+  }
+
+  public static Set<String> setOf(String...items)
+  {
+    if (items.length == 0) {
+      return null;
+    }
+    return new HashSet<>(Arrays.asList(items));
+  }
+
+  public static byte[] toBytes(ObjectMapper jsonMapper, Object obj)
+  {
+    try {
+      return jsonMapper.writeValueAsBytes(obj);
+    }
+    catch (JsonProcessingException e) {
+      throw new ISE("Failed to serialize " + obj.getClass().getSimpleName());
+    }
+  }
+
+  public static <T> T fromBytes(ObjectMapper jsonMapper, byte[] bytes, Class<T> clazz)
+  {
+    try {
+      return jsonMapper.readValue(bytes, clazz);
+    }
+    catch (IOException e) {
+      throw new ISE(e, "Failed to deserialize a " + clazz.getSimpleName());
+    }
+  }
+
+  public static String toString(Object obj)
+  {
+    ObjectMapper jsonMapper = new ObjectMapper();

Review Comment:
   The purpose of this is to save manually generating a `toString()` for methods used only for debugging. There is no way to pass an existing mapper into a `toString()` method, hence the inefficient (but OK-for-debugging) approach to create one each time. Added Javadoc to explain this thinking.
   
   Else, I gotta hand-generate (and maintain) the `toString()` methods for the catalog objects...



##########
server/src/main/java/org/apache/druid/catalog/model/Parameterized.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.druid.catalog.model;
+
+import org.apache.druid.catalog.model.table.ExternalSpec;
+
+import java.util.List;
+import java.util.Map;
+
+public interface Parameterized

Review Comment:
   Changed the name to `ParameterizedDefn` since this parameterizability is an attribute of a definition. (A definition is like a "class" for catalog specs which are like "objects.")



##########
server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.guice.annotations.UnstableApi;
+import org.apache.druid.java.util.common.IAE;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Specification of table columns. Columns have multiple types

Review Comment:
   In the original design, one could mark a datasource as `rollup` with column types `dimension` and `measure`. Now that we removed that idea, all remaining table definitions have only one column type. But, I leave the ability to have multiple types so we can, say, add computed columns (or resurrect the `rollup` thing later.)



##########
server/src/main/java/org/apache/druid/catalog/model/facade/ColumnFacade.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.druid.catalog.model.facade;
+
+import org.apache.druid.catalog.model.ColumnDefn.ResolvedColumn;
+import org.apache.druid.catalog.model.Columns;
+import org.apache.druid.catalog.model.table.DatasourceDefn;
+import org.apache.druid.catalog.model.table.MeasureTypes;
+import org.apache.druid.catalog.model.table.MeasureTypes.MeasureType;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.segment.column.ColumnType;
+
+public class ColumnFacade
+{
+  public static class DatasourceColumnFacade extends ColumnFacade

Review Comment:
   Yes. I would have included more of this layer, since it depends only on the other bits in the PR, but IntelliJ inspections won't let me... I had to jimmy this much using the annotations.



##########
server/src/main/java/org/apache/druid/catalog/model/facade/InputTableFacade.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.druid.catalog.model.facade;
+
+import org.apache.druid.catalog.model.ColumnSpec;
+import org.apache.druid.catalog.model.Columns;
+import org.apache.druid.catalog.model.ResolvedTable;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.column.ColumnType;
+import org.apache.druid.segment.column.RowSignature;
+
+import java.util.List;
+
+public class InputTableFacade extends TableFacade

Review Comment:
   Right! Thanks for point out the use of the old `Input` term: just renamed this to `ExternalTableFacade`.
   
   A facade represents a Java method-based interface to properties stored as maps in the table specs. The facade is for use in Calcite where we don't care about maps and properties: we just want the values.
   
   For now, the only thing this facade handles is the row signature. In the next PR, we'll pull out more of the external table properties. But, I can't add those method to this PR else the build with fail with IntelliJ inspection errors.



##########
server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.joda.time.Period;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class CatalogUtils
+{
+  // Amazing that a parser doesn't already exist...
+  private static final Map<String, Granularity> GRANULARITIES = new HashMap<>();
+
+  static {
+    GRANULARITIES.put("millisecond", Granularities.SECOND);
+    GRANULARITIES.put("second", Granularities.SECOND);
+    GRANULARITIES.put("minute", Granularities.MINUTE);
+    GRANULARITIES.put("5 minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("5 minutes", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("five_minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("10 minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("10 minutes", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("ten_minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("15 minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("15 minutes", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("fifteen_minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("30 minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("30 minutes", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("thirty_minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("hour", Granularities.HOUR);
+    GRANULARITIES.put("6 hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("6 hours", Granularities.SIX_HOUR);
+    GRANULARITIES.put("six_hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("day", Granularities.DAY);
+    GRANULARITIES.put("week", Granularities.WEEK);
+    GRANULARITIES.put("month", Granularities.MONTH);
+    GRANULARITIES.put("quarter", Granularities.QUARTER);
+    GRANULARITIES.put("year", Granularities.YEAR);
+    GRANULARITIES.put("all", Granularities.ALL);
+  }
+
+  public static Granularity toGranularity(String value)
+  {
+    return GRANULARITIES.get(StringUtils.toLowerCase(value));
+  }
+
+  public static int findColumn(List<ColumnSpec> columns, String colName)
+  {
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).name().equals(colName)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static List<String> columnNames(List<ColumnSpec> columns)
+  {
+    return columns
+           .stream()
+           .map(col -> col.name())
+           .collect(Collectors.toList());
+  }
+
+  public static <T extends ColumnSpec> List<T> dropColumns(

Review Comment:
   Used in only one place: so moved there so that the naming makes more sense.



##########
server/src/main/java/org/apache/druid/catalog/model/ObjectDefn.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.catalog.model.Properties.PropertyDefn;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Metadata definition of the metadata objects stored in the catalog. (Yes,
+ * that means that this is meta-meta-data.) Objects consist of a map of
+ * property values (and perhaps other items defined in subclasses.) Each
+ * property is defined by a column metadata object. Objects allow extended
+ * properties which have no definition: the meaning of such properties is
+ * defined elsewhere.
+ */
+public class ObjectDefn
+{
+  private final String name;
+  private final String typeValue;
+  private final Map<String, PropertyDefn> properties;
+
+  public ObjectDefn(
+      final String name,
+      final String typeValue,
+      final List<PropertyDefn> fields
+  )
+  {
+    this.name = name;
+    this.typeValue = typeValue;
+    this.properties = toPropertyMap(fields);
+  }
+
+  protected static Map<String, PropertyDefn> toPropertyMap(final List<PropertyDefn> props)
+  {
+    ImmutableMap.Builder<String, PropertyDefn> builder = ImmutableMap.builder();
+    if (props != null) {
+      for (PropertyDefn prop : props) {
+        builder.put(prop.name(), prop);
+      }
+    }
+    return builder.build();
+  }
+
+  public String name()
+  {
+    return name;
+  }
+
+  /**
+   * The type value is the value of the {@code "type"} field written into the
+   * object's Java or JSON representation. It is akin to the type used by
+   * Jackson.
+   */
+  public String typeValue()
+  {
+    return typeValue;
+  }
+
+  public Map<String, PropertyDefn> properties()
+  {
+    return properties;
+  }
+
+  public PropertyDefn property(String key)
+  {
+    return properties.get(key);
+  }
+
+  /**
+   * Merge the properties for an object using a set of updates in a map. If the
+   * update value is null, then remove the property in the revised set. If the
+   * property is known, use the column definition to merge the values. Else, the
+   * update replaces any existing value.
+   * <p>
+   * This method does not validate the properties, except as needed to do a
+   * merge. A separate validation step is done on the final, merged object.
+   */
+  protected Map<String, Object> mergeProperties(
+      final Map<String, Object> source,
+      final Map<String, Object> update
+  )
+  {
+    if (update == null) {
+      return source;
+    }
+    if (source == null) {
+      return update;
+    }
+    Map<String, Object> merged = new HashMap<>(source);
+    for (Map.Entry<String, Object> entry : update.entrySet()) {
+      if (entry.getValue() == null) {

Review Comment:
   Correct. This is spelled out in the REST API docs in the Apache issue and the Javadoc for this method. This trick avoids the need for an entire REST message just to unset properties. It works because, for no property, is `null` a useful value.



##########
server/src/main/java/org/apache/druid/catalog/model/ColumnSpec.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonInclude.Include;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.guice.annotations.UnstableApi;
+import org.apache.druid.java.util.common.IAE;
+
+import javax.annotation.Nullable;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Specification of table columns. Columns have multiple types
+ * represented via the type field.
+ */
+@UnstableApi
+public class ColumnSpec
+{
+  private final String type;
+  private final String name;
+  private final String sqlType;

Review Comment:
   Sorry, this is a bit tricky. Think of `type` as like a Jackson type: but not quite. All column specs (and all table specs) have the same JSON structure. The `type` refers to the kind of the object (as defined by a `Defn` class.) So, in the original `rollup` datasource type, there was a `dimension` and a `measure` column type; each with a different set of valid properties.
   
   The `sqlType` is the SQL type. It says how SQL should interpret the columns. 
   
   Sorry for the confusion: the Druid standard is to use `type` for the Jackson type, and I didn't want to change that here. I could have used `kind` and `type` (instead of `type` and `sqlType`). I also worried that just a plain `type` name (along with a `kind` field) would lead to the question: is this the Druid or SQL type? And, "what does `kind` mean? Is that the same as the Jackson `type`?)
   
   Naming is hard. I'm open to suggestions.



##########
server/src/main/java/org/apache/druid/catalog/model/Parameterized.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.druid.catalog.model;
+
+import org.apache.druid.catalog.model.table.ExternalSpec;
+
+import java.util.List;
+import java.util.Map;
+
+public interface Parameterized
+{
+  interface ParameterDefn
+  {
+    String name();
+    Class<?> valueClass();
+  }
+
+  class ParameterImpl implements ParameterDefn
+  {
+    private final String name;
+    private final Class<?> type;
+
+    public ParameterImpl(final String name, final Class<?> type)
+    {
+      this.name = name;
+      this.type = type;
+    }
+
+    @Override
+    public String name()
+    {
+      return name;
+    }
+
+    @Override
+    public Class<?> valueClass()
+    {
+      return type;
+    }
+  }
+
+  List<ParameterDefn> parameters();
+  ParameterDefn parameter(String name);
+  ExternalSpec applyParameters(ResolvedTable table, Map<String, Object> parameters);

Review Comment:
   Good eye! In fact, this is one step in the process to create the `ResolvedTable`. The definition class does those other steps and returns the resulting `ResolvedTable`. See, for example, `ExternalTableDefn.ResolvedTable mergeParameters(ResolvedTable table, Map<String, Object> values)`.



##########
server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.joda.time.Period;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class CatalogUtils
+{
+  // Amazing that a parser doesn't already exist...
+  private static final Map<String, Granularity> GRANULARITIES = new HashMap<>();
+
+  static {
+    GRANULARITIES.put("millisecond", Granularities.SECOND);
+    GRANULARITIES.put("second", Granularities.SECOND);
+    GRANULARITIES.put("minute", Granularities.MINUTE);
+    GRANULARITIES.put("5 minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("5 minutes", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("five_minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("10 minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("10 minutes", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("ten_minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("15 minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("15 minutes", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("fifteen_minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("30 minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("30 minutes", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("thirty_minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("hour", Granularities.HOUR);
+    GRANULARITIES.put("6 hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("6 hours", Granularities.SIX_HOUR);
+    GRANULARITIES.put("six_hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("day", Granularities.DAY);
+    GRANULARITIES.put("week", Granularities.WEEK);
+    GRANULARITIES.put("month", Granularities.MONTH);
+    GRANULARITIES.put("quarter", Granularities.QUARTER);
+    GRANULARITIES.put("year", Granularities.YEAR);
+    GRANULARITIES.put("all", Granularities.ALL);
+  }
+
+  public static Granularity toGranularity(String value)
+  {
+    return GRANULARITIES.get(StringUtils.toLowerCase(value));
+  }
+
+  public static int findColumn(List<ColumnSpec> columns, String colName)
+  {
+    for (int i = 0; i < columns.size(); i++) {
+      if (columns.get(i).name().equals(colName)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  public static List<String> columnNames(List<ColumnSpec> columns)
+  {
+    return columns
+           .stream()
+           .map(col -> col.name())
+           .collect(Collectors.toList());
+  }
+
+  public static <T extends ColumnSpec> List<T> dropColumns(
+      final List<T> columns,
+      final List<String> toDrop)
+  {
+    if (toDrop == null || toDrop.isEmpty()) {
+      return columns;
+    }
+    Set<String> drop = new HashSet<String>(toDrop);
+    List<T> revised = new ArrayList<>();
+    for (T col : columns) {
+      if (!drop.contains(col.name())) {
+        revised.add(col);
+      }
+    }
+    return revised;
+  }
+
+  /**
+   * Convert a catalog granularity string to the Druid form. Catalog granularities
+   * are either the usual descriptive strings (in any case), or an ISO period.
+   * For the odd interval, the interval name is also accepted (for the other
+   * intervals, the interval name is the descriptive string).
+   */
+  public static Granularity asDruidGranularity(String value)
+  {
+    if (Strings.isNullOrEmpty(value)) {
+      return Granularities.ALL;
+    }
+    Granularity gran = toGranularity(value);
+    if (gran != null) {
+      return gran;
+    }
+
+    try {
+      return new PeriodGranularity(new Period(value), null, null);
+    }
+    catch (IllegalArgumentException e) {
+      throw new IAE(StringUtils.format("%s is an invalid period string", value));
+    }
+  }
+
+  /**
+   * {@code String}-to-{@code List<String>} conversion. The string can contain zero items,
+   * one items, or a list. The list items are separated by a comma and optional
+   * whitespace.
+   */
+  public static List<String> stringToList(String value)
+  {
+    if (value == null) {
+      return null;
+    }
+    return Arrays.asList(value.split(",\\s*"));
+  }
+
+  public static <T> T safeCast(Object value, Class<T> type, String propertyName)
+  {
+    if (value == null) {
+      return null;
+    }
+    try {
+      return type.cast(value);
+    }
+    catch (ClassCastException e) {
+      throw new IAE("Value [%s] is not valid for property %s, expected type %s",
+          value,
+          propertyName,
+          type.getSimpleName()
+      );
+    }
+  }
+
+  public static <T> T safeGet(Map<String, Object> map, String propertyName, Class<T> type)
+  {
+    return safeCast(map.get(propertyName), type, propertyName);
+  }
+
+  public static String stringListToLines(List<String> lines)
+  {
+    if (lines.isEmpty()) {
+      return "";
+    }
+    return String.join("\n", lines) + "\n";
+  }
+
+  public static Set<String> setOf(String...items)
+  {
+    if (items.length == 0) {
+      return null;
+    }
+    return new HashSet<>(Arrays.asList(items));
+  }
+
+  public static byte[] toBytes(ObjectMapper jsonMapper, Object obj)

Review Comment:
   The key reason for putting these here is that they exploit catalog-specific short-cuts. For the general case, we do need to handle the exception and we should just call the Jackson method directly.
   
   Anyway, moved them to `JacksonUtils`. If they are too specialized for that class, we can move 'em back.



##########
server/src/main/java/org/apache/druid/catalog/model/ColumnDefn.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.druid.catalog.model.Properties.PropertyDefn;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+
+import java.util.List;
+import java.util.Map;
+
+public class ColumnDefn extends ObjectDefn
+{
+  /**
+   * Convenience class that holds a column specification and its corresponding
+   * definition. This allows the spec to be a pure "data object" without knowledge
+   * of the metadata representation given by the column definition.
+   */
+  public static class ResolvedColumn
+  {
+    private final ColumnDefn defn;
+    private final ColumnSpec spec;
+
+    public ResolvedColumn(ColumnDefn defn, ColumnSpec spec)
+    {
+      this.defn = defn;
+      this.spec = spec;
+    }
+
+    public ColumnDefn defn()
+    {
+      return defn;
+    }
+
+    public ColumnSpec spec()
+    {
+      return spec;
+    }
+
+    public ResolvedColumn merge(ColumnSpec update)
+    {
+      return new ResolvedColumn(defn, defn.merge(spec, update));
+    }
+
+    public void validate(ObjectMapper jsonMapper)
+    {
+      defn.validate(spec, jsonMapper);
+    }
+  }
+
+  public ColumnDefn(
+      final String name,
+      final String typeValue,
+      final List<PropertyDefn> fields
+  )
+  {
+    super(name, typeValue, fields);
+  }
+
+  public ColumnSpec merge(ColumnSpec spec, ColumnSpec update)
+  {
+    String updateType = update.type();
+    if (updateType != null && !spec.type().equals(updateType)) {
+      throw new IAE("The update type must be null or [%s]", spec.type());
+    }
+    String revisedType = update.sqlType() == null ? spec.sqlType() : update.sqlType();
+    Map<String, Object> revisedProps = mergeProperties(
+        spec.properties(),
+        update.properties()
+    );
+    return new ColumnSpec(spec.type(), spec.name(), revisedType, revisedProps);

Review Comment:
   Druid does not allow renaming columns: once a name is set, it is fixed. One can remove columns and add new ones. The semantics here reflect that behavior. Also, since we're merging, we need a way to match things up. Since the names must be identical to match, we can't also change the name.
   
   If we did want to allow renaming columns, we'd need a separate REST API (like we have for move and drop columns.)



##########
server/src/main/java/org/apache/druid/catalog/model/table/MeasureTypes.java:
##########
@@ -0,0 +1,216 @@
+/*
+ * 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.druid.catalog.model.table;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.segment.column.ColumnType;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class MeasureTypes
+{
+  public static final String OPTIONAL = "--";
+
+  public enum BaseType
+  {
+    VARCHAR(ColumnType.STRING),
+    BIGINT(ColumnType.LONG),
+    FLOAT(ColumnType.FLOAT),
+    DOUBLE(ColumnType.DOUBLE);
+
+    public final ColumnType nativeType;
+
+    BaseType(ColumnType nativeType)
+    {
+      this.nativeType = nativeType;
+    }
+  }
+
+  public static class MeasureType
+  {
+    public final String name;
+    public final List<BaseType> argTypes;
+    public final String sqlSeedFn;
+    public final String sqlReducerFn;
+    public final String nativeType;
+    public final String nativeAggFn;
+    public final ColumnType storageType;
+    public final String nativeReducerFn;
+
+    public MeasureType(
+        final String name,
+        final List<BaseType> argTypes,
+        final String sqlSeedFn,
+        final String sqlReducerFn,
+        final String nativeType,
+        final ColumnType storageType,
+        final String nativeAggFn,
+        final String nativeReducerFn
+    )
+    {
+      this.name = name;
+      this.argTypes = argTypes == null ? Collections.emptyList() : argTypes;
+      this.sqlSeedFn = sqlSeedFn;
+      this.sqlReducerFn = sqlReducerFn;
+      this.nativeType = nativeType;
+      this.storageType = storageType;
+      this.nativeAggFn = nativeAggFn;
+      this.nativeReducerFn = nativeReducerFn;
+    }
+
+    @Override
+    public String toString()
+    {
+      StringBuilder buf = new StringBuilder()
+          .append(name)
+          .append("(");
+      for (int i = 0; i < argTypes.size(); i++) {
+        if (i > 0) {
+          buf.append(", ");
+        }
+        buf.append(argTypes.get(i).name());
+      }
+      return buf.append(")").toString();
+    }
+  }
+
+  // See: https://druid.apache.org/docs/latest/querying/aggregations.html
+  public static final MeasureType COUNT_TYPE = new MeasureType(
+      "COUNT",
+      null,
+      null,
+      "SUM",
+      "longSum",
+      ColumnType.LONG,
+      "count",
+      "longSum"
+  );
+
+  public static final MeasureType SUM_BIGINT_TYPE = simpleAggType("sum", BaseType.BIGINT);
+  public static final MeasureType SUM_FLOAT_TYPE = simpleAggType("sum", BaseType.FLOAT);
+  public static final MeasureType SUM_DOUBLE_TYPE = simpleAggType("sum", BaseType.DOUBLE);
+  public static final MeasureType MIN_BIGINT_TYPE = simpleAggType("min", BaseType.BIGINT);
+  public static final MeasureType MIN_FLOAT_TYPE = simpleAggType("min", BaseType.FLOAT);
+  public static final MeasureType MIN_DOUBLE_TYPE = simpleAggType("min", BaseType.DOUBLE);
+  public static final MeasureType MAX_BIGINT_TYPE = simpleAggType("max", BaseType.BIGINT);
+  public static final MeasureType MAX_FLOAT_TYPE = simpleAggType("max", BaseType.FLOAT);
+  public static final MeasureType MAX_DOUBLE_TYPE = simpleAggType("max", BaseType.DOUBLE);
+
+  private static MeasureType simpleAggType(String fn, BaseType baseType)
+  {
+    String sqlFn = StringUtils.toUpperCase(fn);
+    String nativeFn = baseType.nativeType.asTypeString() + org.apache.commons.lang3.StringUtils.capitalize(fn);
+    return new MeasureType(
+        sqlFn,
+        Collections.singletonList(baseType),
+        sqlFn,
+        null,
+        sqlFn,
+        baseType.nativeType,
+        nativeFn,
+        nativeFn
+    );
+  }
+
+  private static final List<MeasureType> TYPE_LIST =
+      Arrays.asList(
+          COUNT_TYPE,
+          SUM_BIGINT_TYPE,
+          SUM_FLOAT_TYPE,
+          SUM_DOUBLE_TYPE,
+          MIN_BIGINT_TYPE,
+          MIN_FLOAT_TYPE,
+          MIN_DOUBLE_TYPE,
+          MAX_BIGINT_TYPE,
+          MAX_FLOAT_TYPE,
+          MAX_DOUBLE_TYPE
+      );
+  public static final Map<String, List<MeasureType>> TYPES;
+
+  static {
+    Map<String, List<MeasureType>> map = new HashMap<>();
+    for (MeasureType fn : TYPE_LIST) {
+      List<MeasureType> overloads = map.computeIfAbsent(fn.name, x -> new ArrayList<>());
+      overloads.add(fn);
+    }
+    TYPES = ImmutableMap.<String, List<MeasureType>>builder().putAll(map).build();
+  }
+
+  public static MeasureType parse(String typeStr)
+  {
+    Pattern p = Pattern.compile("(\\w+)(?:\\s*\\((.*)\\))?");

Review Comment:
   This was meant to parse a type name: not an expression. I say "was" because we decided to drop this feature: the code has been removed.



##########
server/src/main/java/org/apache/druid/catalog/model/CatalogUtils.java:
##########
@@ -0,0 +1,242 @@
+/*
+ * 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.druid.catalog.model;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.java.util.common.granularity.Granularity;
+import org.apache.druid.java.util.common.granularity.PeriodGranularity;
+import org.joda.time.Period;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public class CatalogUtils
+{
+  // Amazing that a parser doesn't already exist...
+  private static final Map<String, Granularity> GRANULARITIES = new HashMap<>();
+
+  static {
+    GRANULARITIES.put("millisecond", Granularities.SECOND);
+    GRANULARITIES.put("second", Granularities.SECOND);
+    GRANULARITIES.put("minute", Granularities.MINUTE);
+    GRANULARITIES.put("5 minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("5 minutes", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("five_minute", Granularities.FIVE_MINUTE);
+    GRANULARITIES.put("10 minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("10 minutes", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("ten_minute", Granularities.TEN_MINUTE);
+    GRANULARITIES.put("15 minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("15 minutes", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("fifteen_minute", Granularities.FIFTEEN_MINUTE);
+    GRANULARITIES.put("30 minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("30 minutes", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("thirty_minute", Granularities.THIRTY_MINUTE);
+    GRANULARITIES.put("hour", Granularities.HOUR);
+    GRANULARITIES.put("6 hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("6 hours", Granularities.SIX_HOUR);
+    GRANULARITIES.put("six_hour", Granularities.SIX_HOUR);
+    GRANULARITIES.put("day", Granularities.DAY);
+    GRANULARITIES.put("week", Granularities.WEEK);
+    GRANULARITIES.put("month", Granularities.MONTH);
+    GRANULARITIES.put("quarter", Granularities.QUARTER);
+    GRANULARITIES.put("year", Granularities.YEAR);
+    GRANULARITIES.put("all", Granularities.ALL);
+  }
+
+  public static Granularity toGranularity(String value)
+  {
+    return GRANULARITIES.get(StringUtils.toLowerCase(value));
+  }
+
+  public static int findColumn(List<ColumnSpec> columns, String colName)
+  {
+    for (int i = 0; i < columns.size(); i++) {

Review Comment:
   This is such a simple search that I could not bring myself to invoke a bunch of machinery to get it done. However, happy to change it if the Druid standard is to prefer the machinery.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org