You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ne...@apache.org on 2020/07/17 05:11:45 UTC

[incubator-pinot] branch master updated: DateTimeFieldSpec validation in schema validate (#5706)

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

nehapawar pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 7d2e35b  DateTimeFieldSpec validation in schema validate (#5706)
7d2e35b is described below

commit 7d2e35b2e7071327d52b85e32b915452a049d365
Author: Neha Pawar <ne...@gmail.com>
AuthorDate: Thu Jul 16 22:11:31 2020 -0700

    DateTimeFieldSpec validation in schema validate (#5706)
---
 .../apache/pinot/common/data/FieldSpecTest.java    |   2 +-
 .../org/apache/pinot/common/data/SchemaTest.java   |  18 ++-
 .../api/resources/PinotSchemaRestletResource.java  |  31 +++--
 .../manager/realtime/RealtimeTableDataManager.java |   8 +-
 .../org/apache/pinot/core/util/SchemaUtils.java    | 129 ++++++++++++---------
 .../apache/pinot/core/util/SchemaUtilsTest.java    | 119 +++++++++++++++----
 .../apache/pinot/spi/data/DateTimeFieldSpec.java   |   4 +-
 .../pinot/spi/data/DateTimeFormatPatternSpec.java  |   2 +-
 .../apache/pinot/spi/data/DateTimeFormatSpec.java  |  90 ++++++--------
 .../pinot/spi/data/DateTimeFormatUnitSpec.java     |  14 +--
 .../pinot/spi/data/DateTimeGranularitySpec.java    |  47 +++-----
 .../java/org/apache/pinot/spi/data/Schema.java     |  31 ++---
 .../tools/admin/command/ValidateConfigCommand.java |   4 +-
 .../tools/config/validator/SchemaValidator.java    |   5 +-
 14 files changed, 287 insertions(+), 217 deletions(-)

diff --git a/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java b/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java
index 8063988..dfaad70 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/data/FieldSpecTest.java
@@ -181,7 +181,7 @@ public class FieldSpecTest {
     boolean exceptionActual = false;
     try {
       dateTimeFieldActual = new DateTimeFieldSpec(name, dataType, format, granularity);
-    } catch (IllegalArgumentException e) {
+    } catch (IllegalStateException e) {
       exceptionActual = true;
     }
     Assert.assertEquals(exceptionActual, exceptionExpected);
diff --git a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java
index 0d5cbcf..a0bd761 100644
--- a/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java
+++ b/pinot-common/src/test/java/org/apache/pinot/common/data/SchemaTest.java
@@ -46,16 +46,26 @@ public class SchemaTest {
     Schema schemaToValidate;
 
     schemaToValidate = Schema.fromString(makeSchema(FieldSpec.DataType.LONG, FieldSpec.DataType.STRING, true));
-    Assert.assertTrue(schemaToValidate.validate(LOGGER));
+    schemaToValidate.validate();
 
     schemaToValidate = Schema.fromString(makeSchema(FieldSpec.DataType.BOOLEAN, FieldSpec.DataType.STRING, true));
-    Assert.assertFalse(schemaToValidate.validate(LOGGER));
+    try {
+      schemaToValidate.validate();
+      Assert.fail("Should have failed validation for invalid schema.");
+    } catch (IllegalStateException e) {
+      // expected
+    }
 
     schemaToValidate = Schema.fromString(makeSchema(FieldSpec.DataType.STRING, FieldSpec.DataType.STRING, false));
-    Assert.assertFalse(schemaToValidate.validate(LOGGER));
+    try {
+      schemaToValidate.validate();
+      Assert.fail("Should have failed validation for invalid schema.");
+    } catch (IllegalStateException e) {
+      // expected
+    }
 
     schemaToValidate = Schema.fromString(makeSchema(FieldSpec.DataType.LONG, FieldSpec.DataType.BOOLEAN, false));
-    Assert.assertTrue(schemaToValidate.validate(LOGGER));
+    schemaToValidate.validate();
   }
 
   private String makeSchema(FieldSpec.DataType metricType, FieldSpec.DataType dimensionType, boolean isSingleValue) {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java
index 12841d6..658121a 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSchemaRestletResource.java
@@ -170,9 +170,11 @@ public class PinotSchemaRestletResource {
   @ApiResponses(value = {@ApiResponse(code = 200, message = "Successfully validated schema"), @ApiResponse(code = 400, message = "Missing or invalid request body"), @ApiResponse(code = 500, message = "Internal error")})
   public String validateSchema(FormDataMultiPart multiPart) {
     Schema schema = getSchemaFromMultiPart(multiPart);
-    if (!SchemaUtils.validate(schema, LOGGER)) {
-      throw new ControllerApplicationException(LOGGER, "Invalid schema. Check controller logs",
-          Response.Status.BAD_REQUEST);
+    try {
+      SchemaUtils.validate(schema);
+    } catch (Exception e) {
+      throw new ControllerApplicationException(LOGGER, "Invalid schema: " + schema.getSchemaName(),
+          Response.Status.BAD_REQUEST, e);
     }
     return schema.toPrettyJsonString();
   }
@@ -185,9 +187,11 @@ public class PinotSchemaRestletResource {
       + "from 'GET /schema/{schemaName}'. This allows us to validate schema before apply.")
   @ApiResponses(value = {@ApiResponse(code = 200, message = "Successfully validated schema"), @ApiResponse(code = 400, message = "Missing or invalid request body"), @ApiResponse(code = 500, message = "Internal error")})
   public String validateSchema(Schema schema) {
-    if (!SchemaUtils.validate(schema, LOGGER)) {
-      throw new ControllerApplicationException(LOGGER, "Invalid schema. Check controller logs",
-          Response.Status.BAD_REQUEST);
+    try {
+      SchemaUtils.validate(schema);
+    } catch (Exception e) {
+      throw new ControllerApplicationException(LOGGER, "Invalid schema: " + schema.getSchemaName(),
+          Response.Status.BAD_REQUEST, e);
     }
     return schema.toPrettyJsonString();
   }
@@ -196,12 +200,13 @@ public class PinotSchemaRestletResource {
    * Internal method to add schema
    * @param schema  schema
    * @param override  set to true to override the existing schema with the same name
-   * @return
    */
   private SuccessResponse addSchema(Schema schema, boolean override) {
-    if (!SchemaUtils.validate(schema, LOGGER)) {
-      throw new ControllerApplicationException(LOGGER, "Cannot add invalid schema " + schema.getSchemaName(),
-          Response.Status.BAD_REQUEST);
+    try {
+      SchemaUtils.validate(schema);
+    } catch (Exception e) {
+      throw new ControllerApplicationException(LOGGER, "Cannot add invalid schema: " + schema.getSchemaName(),
+          Response.Status.BAD_REQUEST, e);
     }
 
     try {
@@ -227,9 +232,11 @@ public class PinotSchemaRestletResource {
    * @return
    */
   private SuccessResponse updateSchema(String schemaName, Schema schema, boolean reload) {
-    if (!SchemaUtils.validate(schema, LOGGER)) {
+    try {
+      SchemaUtils.validate(schema);
+    } catch (Exception e) {
       throw new ControllerApplicationException(LOGGER, "Cannot add invalid schema: " + schemaName,
-          Response.Status.BAD_REQUEST);
+          Response.Status.BAD_REQUEST, e);
     }
 
     if (schemaName != null && !schema.getSchemaName().equals(schemaName)) {
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
index 660c0ab..7e18487 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
@@ -397,8 +397,6 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
    * If we add more validations, it may make sense to split this method into multiple validation methods.
    * But then, we are trying to figure out all the invalid cases before we return from this method...
    *
-   * @param schema
-   * @param indexingConfig
    * @return true if schema is valid.
    */
   private boolean isValid(Schema schema, IndexingConfig indexingConfig) {
@@ -417,10 +415,12 @@ public class RealtimeTableDataManager extends BaseTableDataManager {
       }
     }
     // 2. We want to get the schema errors, if any, even if isValid is false;
-    if (!SchemaUtils.validate(schema, _logger)) {
+    try {
+      SchemaUtils.validate(schema);
+    } catch (Exception e) {
+      _logger.error("Caught exception while validating schema: {}", schema.getSchemaName(), e);
       isValid = false;
     }
-
     return isValid;
   }
 }
diff --git a/pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java b/pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
index c16dcad..6691119 100644
--- a/pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
+++ b/pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
@@ -18,15 +18,20 @@
  */
 package org.apache.pinot.core.util;
 
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import org.apache.pinot.core.data.function.FunctionEvaluator;
 import org.apache.pinot.core.data.function.FunctionEvaluatorFactory;
+import org.apache.pinot.spi.data.DateTimeFieldSpec;
+import org.apache.pinot.spi.data.DateTimeFormatSpec;
+import org.apache.pinot.spi.data.DateTimeGranularitySpec;
 import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.data.TimeFieldSpec;
 import org.apache.pinot.spi.data.TimeGranularitySpec;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 /**
@@ -37,67 +42,79 @@ public class SchemaUtils {
   public static final String MAP_KEY_COLUMN_SUFFIX = "__KEYS";
   public static final String MAP_VALUE_COLUMN_SUFFIX = "__VALUES";
 
-  private static final Logger LOGGER = LoggerFactory.getLogger(SchemaUtils.class);
-
-  /**
-   * Validates that for a field spec with transform function, the source column name and destination column name are exclusive
-   * i.e. do not allow using source column name for destination column
-   */
-  public static boolean validate(Schema schema) {
-    return validate(schema, LOGGER);
-  }
-
   /**
    * Validates the following:
-   * 1) for a field spec with transform function, the source column name and destination column name are exclusive
-   * i.e. do not allow using source column name for destination column
-   * 2) Basic schema validations
+   * 1) Checks valid transform function -
+   *   for a field spec with transform function, the source column name and destination column name are exclusive i.e. do not allow using source column name for destination column
+   *   ensure transform function string can be used to create a {@link FunctionEvaluator}
+   * 2) Checks for chained transforms/derived transform - not supported yet
+   * TODO: Transform functions have moved to table config. Once we stop supporting them in schema, remove the validations 1 and 2
+   * 3) Checks valid timeFieldSpec - if incoming and outgoing granularity spec are different a) the names cannot be same b) cannot use SIMPLE_DATE_FORMAT for conversion
+   * 4) Checks valid dateTimeFieldSpecs - checks format and granularity string
+   * 5) Schema validations from {@link Schema#validate}
    */
-  public static boolean validate(Schema schema, Logger logger) {
-    try {
-      for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
-        if (!fieldSpec.isVirtualColumn()) {
-          String column = fieldSpec.getName();
-          String transformFunction = fieldSpec.getTransformFunction();
-          if (transformFunction != null) {
-            FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(fieldSpec);
-            if (functionEvaluator != null) {
-              List<String> arguments = functionEvaluator.getArguments();
-              // output column used as input
-              if (arguments.contains(column)) {
-                logger.error("The arguments of transform function: {}, should not contain the destination column: {}",
-                    transformFunction, column);
-                return false;
-              }
-            }
-          } else if (fieldSpec.getFieldType().equals(FieldSpec.FieldType.TIME)) {
-            TimeFieldSpec timeFieldSpec = (TimeFieldSpec) fieldSpec;
-            TimeGranularitySpec incomingGranularitySpec = timeFieldSpec.getIncomingGranularitySpec();
-            TimeGranularitySpec outgoingGranularitySpec = timeFieldSpec.getOutgoingGranularitySpec();
+  public static void validate(Schema schema) {
+    schema.validate();
 
-            if (!incomingGranularitySpec.equals(outgoingGranularitySpec)) {
-              // different incoming and outgoing spec, but same name
-              if (incomingGranularitySpec.getName().equals(outgoingGranularitySpec.getName())) {
-                logger.error("Cannot convert from incoming field spec:{} to outgoing field spec:{} if name is the same",
-                    incomingGranularitySpec, outgoingGranularitySpec);
-                return false;
-              } else {
-                if (!incomingGranularitySpec.getTimeFormat().equals(TimeGranularitySpec.TimeFormat.EPOCH.toString())
-                    || !outgoingGranularitySpec.getTimeFormat()
-                    .equals(TimeGranularitySpec.TimeFormat.EPOCH.toString())) {
-                  logger.error(
-                      "When incoming and outgoing specs are different, cannot perform time conversion for time format other than EPOCH");
-                  return false;
-                }
-              }
-            }
+    Set<String> transformedColumns = new HashSet<>();
+    Set<String> argumentColumns = new HashSet<>();
+    for (FieldSpec fieldSpec : schema.getAllFieldSpecs()) {
+      if (!fieldSpec.isVirtualColumn()) {
+        String column = fieldSpec.getName();
+        String transformFunction = fieldSpec.getTransformFunction();
+        if (transformFunction != null) {
+          try {
+            List<String> arguments = FunctionEvaluatorFactory.getExpressionEvaluator(fieldSpec).getArguments();
+            Preconditions.checkState(!arguments.contains(column),
+                "The arguments of transform function %s should not contain the destination column %s",
+                transformFunction, column);
+            transformedColumns.add(column);
+            argumentColumns.addAll(arguments);
+          } catch (Exception e) {
+            throw new IllegalStateException(
+                "Exception in getting arguments for transform function '" + transformFunction + "' for column '"
+                    + column + "'", e);
           }
         }
+        if (fieldSpec.getFieldType().equals(FieldSpec.FieldType.TIME)) {
+          validateTimeFieldSpec(fieldSpec);
+        }
+        if (fieldSpec.getFieldType().equals(FieldSpec.FieldType.DATE_TIME)) {
+          validateDateTimeFieldSpec(fieldSpec);
+        }
       }
-    } catch (Exception e) {
-      logger.error("Exception in validating schema {}", schema.getSchemaName(), e);
-      return false;
     }
-    return schema.validate(logger);
+    Preconditions.checkState(Collections.disjoint(transformedColumns, argumentColumns),
+        "Columns: %s are a result of transformations, and cannot be used as arguments to other transform functions",
+        transformedColumns.retainAll(argumentColumns));
+  }
+
+  /**
+   * Checks for valid incoming and outgoing granularity spec in the time field spec
+   */
+  private static void validateTimeFieldSpec(FieldSpec fieldSpec) {
+    TimeFieldSpec timeFieldSpec = (TimeFieldSpec) fieldSpec;
+    TimeGranularitySpec incomingGranularitySpec = timeFieldSpec.getIncomingGranularitySpec();
+    TimeGranularitySpec outgoingGranularitySpec = timeFieldSpec.getOutgoingGranularitySpec();
+
+    if (!incomingGranularitySpec.equals(outgoingGranularitySpec)) {
+      Preconditions.checkState(!incomingGranularitySpec.getName().equals(outgoingGranularitySpec.getName()),
+          "Cannot convert from incoming field spec %s to outgoing field spec %s if name is the same",
+          incomingGranularitySpec, outgoingGranularitySpec);
+
+      Preconditions.checkState(
+          incomingGranularitySpec.getTimeFormat().equals(TimeGranularitySpec.TimeFormat.EPOCH.toString())
+              && outgoingGranularitySpec.getTimeFormat().equals(TimeGranularitySpec.TimeFormat.EPOCH.toString()),
+          "Cannot perform time conversion for time format other than EPOCH. TimeFieldSpec: %s", fieldSpec);
+    }
+  }
+
+  /**
+   * Checks for valid format and granularity string in dateTimeFieldSpec
+   */
+  private static void validateDateTimeFieldSpec(FieldSpec fieldSpec) {
+    DateTimeFieldSpec dateTimeFieldSpec = (DateTimeFieldSpec) fieldSpec;
+    DateTimeFormatSpec.validateFormat(dateTimeFieldSpec.getFormat());
+    DateTimeGranularitySpec.validateGranularity(dateTimeFieldSpec.getGranularity());
   }
 }
diff --git a/pinot-core/src/test/java/org/apache/pinot/core/util/SchemaUtilsTest.java b/pinot-core/src/test/java/org/apache/pinot/core/util/SchemaUtilsTest.java
index fa8a07c..395cce9 100644
--- a/pinot-core/src/test/java/org/apache/pinot/core/util/SchemaUtilsTest.java
+++ b/pinot-core/src/test/java/org/apache/pinot/core/util/SchemaUtilsTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.pinot.core.util;
 
+import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 import org.apache.pinot.spi.data.DateTimeFieldSpec;
 import org.apache.pinot.spi.data.DimensionFieldSpec;
@@ -30,7 +31,7 @@ import org.testng.annotations.Test;
 
 
 /**
- * Tests that the source field names are extracted correctly
+ * Tests schema validations
  */
 public class SchemaUtilsTest {
 
@@ -39,73 +40,149 @@ public class SchemaUtilsTest {
    *  Remove these when we totally stop honoring transform functions in schema
    */
   @Test
-  public void testValidate() {
+  public void testValidateTransformFunctionArguments() {
     Schema pinotSchema;
     // source name used as destination name
     pinotSchema = new Schema();
     DimensionFieldSpec dimensionFieldSpec = new DimensionFieldSpec("dim1", FieldSpec.DataType.STRING, true);
     dimensionFieldSpec.setTransformFunction("Groovy({function}, argument1, dim1, argument3)");
     pinotSchema.addField(dimensionFieldSpec);
-    Assert.assertFalse(SchemaUtils.validate(pinotSchema));
+    try {
+      SchemaUtils.validate(pinotSchema);
+      Assert.fail("Schema validation should have failed.");
+    } catch (IllegalStateException e) {
+      // expected
+    }
 
     pinotSchema = new Schema();
     MetricFieldSpec metricFieldSpec = new MetricFieldSpec("m1", FieldSpec.DataType.LONG);
     metricFieldSpec.setTransformFunction("Groovy({function}, m1, m1)");
     pinotSchema.addField(metricFieldSpec);
-    Assert.assertFalse(SchemaUtils.validate(pinotSchema));
+    checkValidationFails(pinotSchema);
 
     pinotSchema = new Schema();
-    DateTimeFieldSpec dateTimeFieldSpec = new DateTimeFieldSpec("dt1", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS");
+    DateTimeFieldSpec dateTimeFieldSpec =
+        new DateTimeFieldSpec("dt1", FieldSpec.DataType.LONG, "1:HOURS:EPOCH", "1:HOURS");
     dateTimeFieldSpec.setTransformFunction("Groovy({function}, m1, dt1)");
     pinotSchema.addField(dateTimeFieldSpec);
-    Assert.assertFalse(SchemaUtils.validate(pinotSchema));
+    checkValidationFails(pinotSchema);
 
     pinotSchema = new Schema.SchemaBuilder()
         .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "time"), null).build();
     pinotSchema.getFieldSpecFor("time").setTransformFunction("Groovy({function}, time)");
-    Assert.assertFalse(SchemaUtils.validate(pinotSchema));
+    checkValidationFails(pinotSchema);
 
+    // derived transformations
+    pinotSchema = new Schema.SchemaBuilder().addSingleValueDimension("x", FieldSpec.DataType.INT)
+        .addSingleValueDimension("z", FieldSpec.DataType.INT).build();
+    pinotSchema.getFieldSpecFor("x").setTransformFunction("Groovy({y + 10}, y)");
+    pinotSchema.getFieldSpecFor("z").setTransformFunction("Groovy({x*w*20}, x, w)");
+    checkValidationFails(pinotSchema);
+  }
+
+  @Test
+  public void testValidateTimeFieldSpec() {
+    Schema pinotSchema;
     // time field spec using same name for incoming and outgoing
     pinotSchema = new Schema.SchemaBuilder()
         .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "time"),
             new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "time")).build();
-    Assert.assertFalse(SchemaUtils.validate(pinotSchema));
+    checkValidationFails(pinotSchema);
 
     // time field spec using SIMPLE_DATE_FORMAT, not allowed when conversion is needed
     pinotSchema = new Schema.SchemaBuilder()
         .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "incoming"),
             new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS,
                 TimeGranularitySpec.TimeFormat.SIMPLE_DATE_FORMAT.toString(), "outgoing")).build();
-    Assert.assertFalse(SchemaUtils.validate(pinotSchema));
+    checkValidationFails(pinotSchema);
+
+    // valid time field spec
+    pinotSchema = new Schema.SchemaBuilder()
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "incoming"),
+            new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "outgoing")).build();
+    SchemaUtils.validate(pinotSchema);
+  }
 
+  @Test
+  public void testGroovyFunctionSyntax() {
+    Schema pinotSchema;
     // incorrect groovy function syntax
     pinotSchema = new Schema();
-    dimensionFieldSpec = new DimensionFieldSpec("dim1", FieldSpec.DataType.STRING, true);
+
+    DimensionFieldSpec dimensionFieldSpec = new DimensionFieldSpec("dim1", FieldSpec.DataType.STRING, true);
     dimensionFieldSpec.setTransformFunction("Groovy(function, argument3)");
     pinotSchema.addField(dimensionFieldSpec);
-    Assert.assertFalse(SchemaUtils.validate(pinotSchema));
+    checkValidationFails(pinotSchema);
 
     // valid schema, empty arguments
     pinotSchema = new Schema();
+
     dimensionFieldSpec = new DimensionFieldSpec("dim1", FieldSpec.DataType.STRING, true);
     dimensionFieldSpec.setTransformFunction("Groovy({function})");
     pinotSchema.addField(dimensionFieldSpec);
-    Assert.assertTrue(SchemaUtils.validate(pinotSchema));
+    SchemaUtils.validate(pinotSchema);
 
     // valid schema
-    pinotSchema = new Schema.SchemaBuilder()
-        .addSingleValueDimension("dim1", FieldSpec.DataType.STRING)
+    pinotSchema = new Schema.SchemaBuilder().addSingleValueDimension("dim1", FieldSpec.DataType.STRING)
         .addMetric("m1", FieldSpec.DataType.LONG)
-        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "time"), null)
-        .build();
+        .addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "time"), null).build();
     pinotSchema.getFieldSpecFor("dim1").setTransformFunction("Groovy({function}, argument1, argument2, argument3)");
     pinotSchema.getFieldSpecFor("m1").setTransformFunction("Groovy({function}, m2, m3)");
     pinotSchema.getFieldSpecFor("time").setTransformFunction("Groovy({function}, millis)");
-    Assert.assertTrue(SchemaUtils.validate(pinotSchema));
+    SchemaUtils.validate(pinotSchema);
+  }
 
-    // valid time field spec
-    pinotSchema = new Schema.SchemaBuilder().addTime(new TimeGranularitySpec(FieldSpec.DataType.LONG, TimeUnit.MILLISECONDS, "incoming"),
-            new TimeGranularitySpec(FieldSpec.DataType.INT, TimeUnit.DAYS, "outgoing")).build();
-    Assert.assertTrue(SchemaUtils.validate(pinotSchema));
+  @Test
+  public void testDateTimeFieldSpec()
+      throws IOException {
+    Schema pinotSchema;
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"x:HOURS:EPOCH\",\"granularity\":\"1:HOURS\"}]}");
+    checkValidationFails(pinotSchema);
+
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"1:DUMMY:EPOCH\",\"granularity\":\"1:HOURS\"}]}");
+    checkValidationFails(pinotSchema);
+
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"1:HOURS:DUMMY\",\"granularity\":\"1:HOURS\"}]}");
+    checkValidationFails(pinotSchema);
+
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"1:HOURS:EPOCH\",\"granularity\":\"x:HOURS\"}]}");
+    checkValidationFails(pinotSchema);
+
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"1:HOURS:EPOCH\",\"granularity\":\"1:DUMMY\"}]}");
+    checkValidationFails(pinotSchema);
+
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"1:DAYS:SIMPLE_DATE_FORMAT\",\"granularity\":\"1:DAYS\"}]}");
+    checkValidationFails(pinotSchema);
+
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"1:HOURS:EPOCH\",\"granularity\":\"1:HOURS\"}]}");
+    SchemaUtils.validate(pinotSchema);
+
+    pinotSchema = Schema.fromString(
+        "{\"schemaName\":\"testSchema\"," + "\"dimensionFieldSpecs\":[ {\"name\":\"dim1\",\"dataType\":\"STRING\"}],"
+            + "\"dateTimeFieldSpecs\":[{\"name\":\"dt1\",\"dataType\":\"INT\",\"format\":\"1:DAYS:SIMPLE_DATE_FORMAT:yyyyMMdd\",\"granularity\":\"1:DAYS\"}]}");
+    SchemaUtils.validate(pinotSchema);
+  }
+
+  private void checkValidationFails(Schema pinotSchema) {
+    try {
+      SchemaUtils.validate(pinotSchema);
+      Assert.fail("Schema validation should have failed.");
+    } catch (IllegalStateException e) {
+      // expected
+    }
   }
 }
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java
index 1fe75a4..3ec92b9 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFieldSpec.java
@@ -70,8 +70,8 @@ public final class DateTimeFieldSpec extends FieldSpec {
     super(name, dataType, true);
     Preconditions.checkNotNull(name);
     Preconditions.checkNotNull(dataType);
-    Preconditions.checkArgument(DateTimeFormatSpec.isValidFormat(format));
-    Preconditions.checkArgument(DateTimeGranularitySpec.isValidGranularity(granularity));
+    DateTimeFormatSpec.validateFormat(format);
+    DateTimeGranularitySpec.validateGranularity(granularity);
 
     _format = format;
     _granularity = granularity;
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatPatternSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatPatternSpec.java
index 3737a76..e91c257 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatPatternSpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatPatternSpec.java
@@ -39,7 +39,7 @@ public class DateTimeFormatPatternSpec {
   public static final DateTimeZone DEFAULT_DATETIMEZONE = DateTimeZone.UTC;
   public static final Locale DEFAULT_LOCALE = Locale.ENGLISH;
 
-  private TimeFormat _timeFormat;
+  private final TimeFormat _timeFormat;
   private String _sdfPattern = null;
   private DateTimeZone _dateTimeZone = DEFAULT_DATETIMEZONE;
   private transient DateTimeFormatter _dateTimeFormatter;
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatSpec.java
index e9d9924..f760778 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatSpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatSpec.java
@@ -32,14 +32,7 @@ import org.joda.time.format.DateTimeFormatter;
  */
 public class DateTimeFormatSpec {
 
-  public static final String FORMAT_TOKENS_ERROR_STR = "format must be of pattern size:timeunit:timeformat(:pattern)";
-  public static final String FORMAT_PATTERN_ERROR_STR =
-      "format must be of format [0-9]+:<TimeUnit>:<TimeFormat>(:pattern)";
-  public static final String TIME_FORMAT_ERROR_STR =
-      "format must be of format [0-9]+:<TimeUnit>:EPOCH or [0-9]+:<TimeUnit>:SIMPLE_DATE_FORMAT:<format>";
-
   public static final String NUMBER_REGEX = "[1-9][0-9]*";
-
   public static final String COLON_SEPARATOR = ":";
 
   /* DateTimeFieldSpec format is of format size:timeUnit:timeformat:pattern tz(timezone)
@@ -51,36 +44,31 @@ public class DateTimeFormatSpec {
   public static final int MIN_FORMAT_TOKENS = 3;
   public static final int MAX_FORMAT_TOKENS = 4;
 
-  private String _format;
-  private int _size;
-  private DateTimeFormatUnitSpec _unitSpec;
-  private DateTimeFormatPatternSpec _patternSpec;
+  private final String _format;
+  private final int _size;
+  private final DateTimeFormatUnitSpec _unitSpec;
+  private final DateTimeFormatPatternSpec _patternSpec;
 
   public DateTimeFormatSpec(String format) {
     _format = format;
-    if (isValidFormat(format)) {
-      String[] formatTokens = format.split(COLON_SEPARATOR, MAX_FORMAT_TOKENS);
-      _size = Integer.valueOf(formatTokens[FORMAT_SIZE_POSITION]);
-      _unitSpec = new DateTimeFormatUnitSpec(formatTokens[FORMAT_UNIT_POSITION]);
-      if (formatTokens.length == MAX_FORMAT_TOKENS) {
-        _patternSpec = new DateTimeFormatPatternSpec(formatTokens[FORMAT_TIMEFORMAT_POSITION],
-            formatTokens[FORMAT_PATTERN_POSITION]);
-      } else {
-        _patternSpec = new DateTimeFormatPatternSpec(formatTokens[FORMAT_TIMEFORMAT_POSITION], null);
-      }
+    validateFormat(format);
+    String[] formatTokens = format.split(COLON_SEPARATOR, MAX_FORMAT_TOKENS);
+    _size = Integer.parseInt(formatTokens[FORMAT_SIZE_POSITION]);
+    _unitSpec = new DateTimeFormatUnitSpec(formatTokens[FORMAT_UNIT_POSITION]);
+    if (formatTokens.length == MAX_FORMAT_TOKENS) {
+      _patternSpec = new DateTimeFormatPatternSpec(formatTokens[FORMAT_TIMEFORMAT_POSITION],
+          formatTokens[FORMAT_PATTERN_POSITION]);
+    } else {
+      _patternSpec = new DateTimeFormatPatternSpec(formatTokens[FORMAT_TIMEFORMAT_POSITION], null);
     }
   }
 
   /**
    * Constructs a dateTimeSpec format, given the components of a format
-   * @param columnSize
-   * @param columnUnit
-   * @param columnTimeFormat
-   * @return
    */
   public DateTimeFormatSpec(int columnSize, String columnUnit, String columnTimeFormat) {
     _format = Joiner.on(COLON_SEPARATOR).join(columnSize, columnUnit, columnTimeFormat);
-    isValidFormat(_format);
+    validateFormat(_format);
 
     _size = columnSize;
     _unitSpec = new DateTimeFormatUnitSpec(columnUnit);
@@ -89,15 +77,11 @@ public class DateTimeFormatSpec {
 
   /**
    * Constructs a dateTimeSpec format, given the components of a format
-   * @param columnSize
-   * @param columnUnit
-   * @param columnTimeFormat
    * @param sdfPattern and tz
-   * @return
    */
   public DateTimeFormatSpec(int columnSize, String columnUnit, String columnTimeFormat, String sdfPattern) {
     _format = Joiner.on(COLON_SEPARATOR).join(columnSize, columnUnit, columnTimeFormat, sdfPattern);
-    isValidFormat(_format);
+    validateFormat(_format);
 
     _size = columnSize;
     _unitSpec = new DateTimeFormatUnitSpec(columnUnit);
@@ -140,7 +124,7 @@ public class DateTimeFormatSpec {
    * <ul>
    * <li>Given a timestamp in millis, convert it to the given format
    * This method should not do validation of outputGranularity.
-   * The validation should be handled by caller using {@link #isValidFormat(String)}</li>
+   * The validation should be handled by caller using {@link #validateFormat}</li>
    * <ul>
    * <li>1) given dateTimeColumnValueMS = 1498892400000 and format=1:HOURS:EPOCH,
    * dateTimeSpec.fromMillis(1498892400000) = 416359 (i.e. dateTimeColumnValueMS/(1000*60*60))</li>
@@ -150,16 +134,13 @@ public class DateTimeFormatSpec {
    * format=1:DAYS:SIMPLE_DATE_FORMAT:yyyyMMdd, dateTimeSpec.fromMillis(1498892400000) = 20170701</li>
    * </ul>
    * </ul>
-   * @param dateTimeColumnValueMS
-   * @param toFormat - the format in which to convert the millis value
    * @param type - type of return value (can be int/long or string depending on time format)
    * @return dateTime column value in dateTimeFieldSpec
    */
   public <T extends Object> T fromMillisToFormat(Long dateTimeColumnValueMS, Class<T> type) {
     Preconditions.checkNotNull(dateTimeColumnValueMS);
 
-    Object dateTimeColumnValue = null;
-
+    Object dateTimeColumnValue;
     if (_patternSpec.getTimeFormat().equals(TimeFormat.EPOCH)) {
       dateTimeColumnValue = _unitSpec.getTimeUnit().convert(dateTimeColumnValueMS, TimeUnit.MILLISECONDS) / _size;
     } else {
@@ -172,7 +153,7 @@ public class DateTimeFormatSpec {
    * <ul>
    * <li>Convert a time value in a format, to millis.
    * This method should not do validation of outputGranularity.
-   * The validation should be handled by caller using {@link #isValidFormat(String)}</li>
+   * The validation should be handled by caller using {@link #validateFormat}</li>
    * <ul>
    * <li>1) given dateTimeColumnValue = 416359 and format=1:HOURS:EPOCH
    * dateTimeSpec.toMillis(416359) = 1498892400000 (i.e. timeColumnValue*60*60*1000)</li>
@@ -183,41 +164,44 @@ public class DateTimeFormatSpec {
    * </ul>
    * <ul>
    * @param dateTimeColumnValue - datetime Column value to convert to millis
-   * @param fromFormat - the format in which the date time column value is expressed
    * @return datetime value in millis
    */
   public Long fromFormatToMillis(Object dateTimeColumnValue) {
     Preconditions.checkNotNull(dateTimeColumnValue);
 
-    Long timeColumnValueMS = 0L;
-
+    long timeColumnValueMS;
     if (_patternSpec.getTimeFormat().equals(TimeFormat.EPOCH)) {
       timeColumnValueMS = TimeUnit.MILLISECONDS.convert((Long) dateTimeColumnValue * _size, _unitSpec.getTimeUnit());
     } else {
       timeColumnValueMS = _patternSpec.getDateTimeFormatter().parseMillis(String.valueOf(dateTimeColumnValue));
     }
-
     return timeColumnValueMS;
   }
 
-  public static boolean isValidFormat(String format) {
-    Preconditions.checkNotNull(format);
-
+  /**
+   * Validates the format string in the dateTimeFieldSpec
+   */
+  public static void validateFormat(String format) {
+    Preconditions.checkNotNull(format, "Format string in dateTimeFieldSpec must not be null");
     String[] formatTokens = format.split(COLON_SEPARATOR, MAX_FORMAT_TOKENS);
-    Preconditions.checkArgument(formatTokens.length == MIN_FORMAT_TOKENS || formatTokens.length == MAX_FORMAT_TOKENS,
-        FORMAT_TOKENS_ERROR_STR);
+    Preconditions.checkState(formatTokens.length >= MIN_FORMAT_TOKENS && formatTokens.length <= MAX_FORMAT_TOKENS,
+        "Incorrect format: %s. Must be of format 'size:timeunit:timeformat(:pattern)'", format);
+    Preconditions.checkState(formatTokens[FORMAT_SIZE_POSITION].matches(NUMBER_REGEX),
+        "Incorrect format size: %s in format: %s. Must be of format '[0-9]+:<TimeUnit>:<TimeFormat>(:pattern)'",
+        formatTokens[FORMAT_SIZE_POSITION], format);
+
+    DateTimeFormatUnitSpec.validateUnitSpec(formatTokens[FORMAT_UNIT_POSITION]);
 
-    Preconditions.checkArgument(formatTokens[FORMAT_SIZE_POSITION].matches(NUMBER_REGEX), FORMAT_PATTERN_ERROR_STR);
-    Preconditions.checkArgument(DateTimeFormatUnitSpec.isValidUnitSpec(formatTokens[FORMAT_UNIT_POSITION]));
     if (formatTokens.length == MIN_FORMAT_TOKENS) {
-      Preconditions.checkArgument(formatTokens[FORMAT_TIMEFORMAT_POSITION].equals(TimeFormat.EPOCH.toString()),
-          TIME_FORMAT_ERROR_STR);
+      Preconditions.checkState(formatTokens[FORMAT_TIMEFORMAT_POSITION].equals(TimeFormat.EPOCH.toString()),
+          "Incorrect format type: %s in format: %s. Must be of '[0-9]+:<TimeUnit>:EPOCH'",
+          formatTokens[FORMAT_TIMEFORMAT_POSITION], format);
     } else {
       Preconditions
-          .checkArgument(formatTokens[FORMAT_TIMEFORMAT_POSITION].equals(TimeFormat.SIMPLE_DATE_FORMAT.toString()),
-              TIME_FORMAT_ERROR_STR);
+          .checkState(formatTokens[FORMAT_TIMEFORMAT_POSITION].equals(TimeFormat.SIMPLE_DATE_FORMAT.toString()),
+              "Incorrect format type: %s in format: %s. Must be of '[0-9]+:<TimeUnit>:SIMPLE_DATE_FORMAT:pattern'",
+              formatTokens[FORMAT_TIMEFORMAT_POSITION], format);
     }
-    return true;
   }
 
   @Override
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatUnitSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatUnitSpec.java
index 9c1f4b5..56c11e0 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatUnitSpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeFormatUnitSpec.java
@@ -18,6 +18,7 @@
  */
 package org.apache.pinot.spi.data;
 
+import com.google.common.base.Preconditions;
 import java.util.concurrent.TimeUnit;
 import org.apache.commons.lang3.EnumUtils;
 import org.apache.pinot.spi.utils.EqualityUtils;
@@ -101,9 +102,7 @@ public class DateTimeFormatUnitSpec {
   private DateTimeTransformUnit _dateTimeTransformUnit = null;
 
   public DateTimeFormatUnitSpec(String unit) {
-    if (!isValidUnitSpec(unit)) {
-      throw new IllegalArgumentException("Unit must belong to enum TimeUnit or DateTimeTransformUnit");
-    }
+    validateUnitSpec(unit);
     if (EnumUtils.isValidEnum(TimeUnit.class, unit)) {
       _timeUnit = TimeUnit.valueOf(unit);
     }
@@ -120,11 +119,10 @@ public class DateTimeFormatUnitSpec {
     return _dateTimeTransformUnit;
   }
 
-  public static boolean isValidUnitSpec(String unit) {
-    if (EnumUtils.isValidEnum(TimeUnit.class, unit) || EnumUtils.isValidEnum(DateTimeTransformUnit.class, unit)) {
-      return true;
-    }
-    return false;
+  public static void validateUnitSpec(String unit) {
+    Preconditions.checkState(
+        EnumUtils.isValidEnum(TimeUnit.class, unit) || EnumUtils.isValidEnum(DateTimeTransformUnit.class, unit),
+        "Unit: %s must belong to enum TimeUnit or DateTimeTransformUnit", unit);
   }
 
   @Override
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeGranularitySpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeGranularitySpec.java
index 20ce8bd..e195f8e 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeGranularitySpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/DateTimeGranularitySpec.java
@@ -30,8 +30,6 @@ import org.apache.pinot.spi.utils.EqualityUtils;
  */
 public class DateTimeGranularitySpec {
 
-  public static final String GRANULARITY_TOKENS_ERROR_STR = "granularity must be of format size:timeunit";
-  public static final String GRANULARITY_PATTERN_ERROR_STR = "granularity must be of format [0-9]+:<TimeUnit>";
   public static final String NUMBER_REGEX = "[1-9][0-9]*";
 
   public static final String COLON_SEPARATOR = ":";
@@ -41,34 +39,29 @@ public class DateTimeGranularitySpec {
   public static final int GRANULARITY_UNIT_POSITION = 1;
   public static final int MAX_GRANULARITY_TOKENS = 2;
 
-  private String _granularity;
-  private int _size;
-  private TimeUnit _timeUnit;
+  private final String _granularity;
+  private final int _size;
+  private final TimeUnit _timeUnit;
 
   /**
    * Constructs a dateTimeGranularitySpec granularity from a string
-   * @param granularity
-   * @return
    */
   public DateTimeGranularitySpec(String granularity) {
+    validateGranularity(granularity);
     _granularity = granularity;
-    isValidGranularity(granularity);
     String[] granularityTokens = _granularity.split(COLON_SEPARATOR);
-    _size = Integer.valueOf(granularityTokens[GRANULARITY_SIZE_POSITION]);
+    _size = Integer.parseInt(granularityTokens[GRANULARITY_SIZE_POSITION]);
     _timeUnit = TimeUnit.valueOf(granularityTokens[GRANULARITY_UNIT_POSITION]);
   }
 
   /**
    * Constructs a dateTimeGranularitySpec granularity given the components of a granularity
-   * @param columnSize
-   * @param columnUnit
-   * @return
    */
   public DateTimeGranularitySpec(int columnSize, TimeUnit columnUnit) {
     _granularity = Joiner.on(COLON_SEPARATOR).join(columnSize, columnUnit);
+    validateGranularity(_granularity);
     _size = columnSize;
     _timeUnit = columnUnit;
-    isValidGranularity(_granularity);
   }
 
   public String getGranularity() {
@@ -87,14 +80,13 @@ public class DateTimeGranularitySpec {
    * <ul>
    * <li>Convert a granularity to millis.
    * This method should not do validation of outputGranularity.
-   * The validation should be handled by caller using {@link #isValidGranularity(String)}</li>
+   * The validation should be handled by caller using {@link #validateGranularity}</li>
    * <ul>
    * <li>1) granularityToMillis(1:HOURS) = 3600000 (60*60*1000)</li>
    * <li>2) granularityToMillis(1:MILLISECONDS) = 1</li>
    * <li>3) granularityToMillis(15:MINUTES) = 900000 (15*60*1000)</li>
    * </ul>
    * </ul>
-   * @return
    */
   public Long granularityToMillis() {
     return TimeUnit.MILLISECONDS.convert(_size, _timeUnit);
@@ -102,19 +94,19 @@ public class DateTimeGranularitySpec {
 
   /**
    * Check correctness of granularity of {@link DateTimeFieldSpec}
-   * @param granularity
-   * @return
    */
-  public static boolean isValidGranularity(String granularity) {
-    Preconditions.checkNotNull(granularity);
-    String[] granularityTokens = granularity.split(COLON_SEPARATOR);
-    Preconditions.checkArgument(granularityTokens.length == MAX_GRANULARITY_TOKENS, GRANULARITY_TOKENS_ERROR_STR);
-    Preconditions.checkArgument(granularityTokens[GRANULARITY_SIZE_POSITION].matches(NUMBER_REGEX),
-        GRANULARITY_PATTERN_ERROR_STR);
-    Preconditions.checkArgument(EnumUtils.isValidEnum(TimeUnit.class, granularityTokens[GRANULARITY_UNIT_POSITION]),
-        GRANULARITY_PATTERN_ERROR_STR);
+  public static void validateGranularity(String granularity) {
+    Preconditions.checkNotNull(granularity, "Granularity string in dateTimeFieldSpec must not be null");
 
-    return true;
+    String[] granularityTokens = granularity.split(COLON_SEPARATOR);
+    Preconditions.checkState(granularityTokens.length == MAX_GRANULARITY_TOKENS,
+        "Incorrect granularity: %s. Must be of format 'size:timeunit'", granularity);
+    Preconditions.checkState(granularityTokens[GRANULARITY_SIZE_POSITION].matches(NUMBER_REGEX),
+        "Incorrect granularity size: %s. Must be of format '[0-9]+:<TimeUnit>'",
+        granularityTokens[GRANULARITY_SIZE_POSITION]);
+    Preconditions.checkState(EnumUtils.isValidEnum(TimeUnit.class, granularityTokens[GRANULARITY_UNIT_POSITION]),
+        "Incorrect granularity size: %s. Must be of format '[0-9]+:<TimeUnit>'",
+        granularityTokens[GRANULARITY_SIZE_POSITION]);
   }
 
   @Override
@@ -134,7 +126,6 @@ public class DateTimeGranularitySpec {
 
   @Override
   public int hashCode() {
-    int result = EqualityUtils.hashCodeOf(_granularity);
-    return result;
+    return EqualityUtils.hashCodeOf(_granularity);
   }
 }
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java
index fb6330e..720f075 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/Schema.java
@@ -403,16 +403,8 @@ public final class Schema {
    *   <li>For dimension, time, date time fields, support {@link DataType}: INT, LONG, FLOAT, DOUBLE, STRING</li>
    *   <li>For non-derived metric fields, support {@link DataType}: INT, LONG, FLOAT, DOUBLE</li>
    * </ul>
-   *
-   * @param ctxLogger Logger used to log the message (if null, the current class logger is used)
-   * @return Whether schema is valid
    */
-  public boolean validate(Logger ctxLogger) {
-    if (ctxLogger == null) {
-      ctxLogger = LOGGER;
-    }
-
-    // Log ALL the schema errors that may be present.
+  public void validate() {
     for (FieldSpec fieldSpec : _fieldSpecMap.values()) {
       FieldType fieldType = fieldSpec.getFieldType();
       DataType dataType = fieldSpec.getDataType();
@@ -430,8 +422,8 @@ public final class Schema {
             case BYTES:
               break;
             default:
-              ctxLogger.info("Unsupported data type: {} in DIMENSION/TIME field: {}", dataType, fieldName);
-              return false;
+              throw new IllegalStateException(
+                  "Unsupported data type: " + dataType + " in DIMENSION/TIME field: " + fieldName);
           }
           break;
         case METRIC:
@@ -443,8 +435,7 @@ public final class Schema {
             case BYTES:
               break;
             default:
-              ctxLogger.info("Unsupported data type: {} in METRIC field: {}", dataType, fieldName);
-              return false;
+              throw new IllegalStateException("Unsupported data type: " + dataType + " in METRIC field: " + fieldName);
           }
           break;
         case COMPLEX:
@@ -454,16 +445,12 @@ public final class Schema {
             case LIST:
               break;
             default:
-              ctxLogger.info("Unsupported data type: {} in COMPLEX field: {}", dataType, fieldName);
-              return false;
+              throw new IllegalStateException("Unsupported data type: " + dataType + " in COMPLEX field: " + fieldName);
           }
         default:
-          ctxLogger.info("Unsupported field type: {} for field: {}", dataType, fieldName);
-          return false;
+          throw new IllegalStateException("Unsupported data type: " + dataType + " for field: " + fieldName);
       }
     }
-
-    return true;
   }
 
   public static class SchemaBuilder {
@@ -572,8 +559,10 @@ public final class Schema {
     }
 
     public Schema build() {
-      if (!_schema.validate(LOGGER)) {
-        throw new RuntimeException("Invalid schema");
+      try {
+        _schema.validate();
+      } catch (Exception e) {
+        throw new RuntimeException("Invalid schema", e);
       }
       return _schema;
     }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/ValidateConfigCommand.java b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/ValidateConfigCommand.java
index de4533c..92711c6 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/ValidateConfigCommand.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/ValidateConfigCommand.java
@@ -142,9 +142,7 @@ public class ValidateConfigCommand extends AbstractBaseCommand implements Comman
       try {
         ZNRecord record = _helixPropertyStore.get(SCHEMA_PATH + "/" + schemaName, null, 0);
         Schema schema = SchemaUtils.fromZNRecord(record);
-        if (!SchemaValidator.validate(schema)) {
-          LOGGER.error("    Schema validation failed for schema: \"{}\"", schemaName);
-        }
+        SchemaValidator.validate(schema);
       } catch (Exception e) {
         LOGGER.error("    Caught exception while validating schema: \"{}\"", schemaName, e);
       }
diff --git a/pinot-tools/src/main/java/org/apache/pinot/tools/config/validator/SchemaValidator.java b/pinot-tools/src/main/java/org/apache/pinot/tools/config/validator/SchemaValidator.java
index 610c1c5..e33ebbb 100644
--- a/pinot-tools/src/main/java/org/apache/pinot/tools/config/validator/SchemaValidator.java
+++ b/pinot-tools/src/main/java/org/apache/pinot/tools/config/validator/SchemaValidator.java
@@ -26,8 +26,7 @@ public class SchemaValidator {
   private SchemaValidator() {
   }
 
-  public static boolean validate(Schema schema) {
-    // TODO: ADD MORE VALIDATIONS.
-    return SchemaUtils.validate(schema);
+  public static void validate(Schema schema) {
+    SchemaUtils.validate(schema);
   }
 }


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