You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/07/13 16:59:11 UTC

[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #5642: Add more schema validations

mcvsubbu commented on a change in pull request #5642:
URL: https://github.com/apache/incubator-pinot/pull/5642#discussion_r453785181



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,134 @@
 
   private static final Logger LOGGER = LoggerFactory.getLogger(SchemaUtils.class);
 
+  private static final SqlAbstractParserImpl.Metadata SQL_PARSER_METADATA = SqlParser.create("",
+      SqlParser.configBuilder().setConformance(SqlConformanceEnum.BABEL).setParserFactory(SqlBabelParserImpl.FACTORY)
+          .build()).getMetadata();
+
   /**
-   * 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
+   * Validates the following:
+   * 1) Checks if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
    */
   public static boolean validate(Schema schema) {
-    return validate(schema, LOGGER);
+    return validate(schema, true, 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 if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
+   *
+   * @param validateFieldNames if false, does not validate field names. This is to prevent validation failing on existing schemas with invalid field names during a schema update
    */
-  public static boolean validate(Schema schema, Logger logger) {
+  public static boolean validate(Schema schema, boolean validateFieldNames, @Nullable 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;
-              }
+          if (validateFieldNames && !isValidFieldName(fieldSpec)) {
+            return false;
+          }
+          if (!isValidTransformFunction(fieldSpec)) {

Review comment:
       Pass the logger to all the isValidXXX methods so that you can log the message to the right logger.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,134 @@
 
   private static final Logger LOGGER = LoggerFactory.getLogger(SchemaUtils.class);
 
+  private static final SqlAbstractParserImpl.Metadata SQL_PARSER_METADATA = SqlParser.create("",

Review comment:
       I would not take a sql dependency here. I think it is cleaner to separate the query language used from Pinot schema. No matter which query language we use, there will be some reserved words.
   I would have a sql init call into schema utils init to set the reserved words here.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,134 @@
 
   private static final Logger LOGGER = LoggerFactory.getLogger(SchemaUtils.class);
 
+  private static final SqlAbstractParserImpl.Metadata SQL_PARSER_METADATA = SqlParser.create("",
+      SqlParser.configBuilder().setConformance(SqlConformanceEnum.BABEL).setParserFactory(SqlBabelParserImpl.FACTORY)
+          .build()).getMetadata();
+
   /**
-   * 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
+   * Validates the following:
+   * 1) Checks if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
    */
   public static boolean validate(Schema schema) {
-    return validate(schema, LOGGER);
+    return validate(schema, true, 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 if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
+   *
+   * @param validateFieldNames if false, does not validate field names. This is to prevent validation failing on existing schemas with invalid field names during a schema update
    */
-  public static boolean validate(Schema schema, Logger logger) {
+  public static boolean validate(Schema schema, boolean validateFieldNames, @Nullable 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;
-              }
+          if (validateFieldNames && !isValidFieldName(fieldSpec)) {
+            return false;
+          }
+          if (!isValidTransformFunction(fieldSpec)) {
+            return false;
+          }
+          FieldSpec.FieldType fieldType = fieldSpec.getFieldType();
+          if (fieldType.equals(FieldSpec.FieldType.DATE_TIME)) {
+            if (!isValidDateTimeFieldSpec(fieldSpec)) {
+              return false;
             }
-          } else if (fieldSpec.getFieldType().equals(FieldSpec.FieldType.TIME)) {
-            TimeFieldSpec timeFieldSpec = (TimeFieldSpec) fieldSpec;
-            TimeGranularitySpec incomingGranularitySpec = timeFieldSpec.getIncomingGranularitySpec();
-            TimeGranularitySpec outgoingGranularitySpec = timeFieldSpec.getOutgoingGranularitySpec();
-
-            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;
-                }
-              }
+          } else if (fieldType.equals(FieldSpec.FieldType.TIME)) {
+            if (!isValidTimeFieldSpec(fieldSpec)) {
+              return false;
             }
           }
         }
       }
     } catch (Exception e) {
-      logger.error("Exception in validating schema {}", schema.getSchemaName(), e);
+      LOGGER.error("Exception in validating schema {}", schema.getSchemaName(), e);

Review comment:
       ```suggestion
         logger.error("Exception in validating schema {}", schema.getSchemaName(), e);
   ```

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,134 @@
 
   private static final Logger LOGGER = LoggerFactory.getLogger(SchemaUtils.class);
 
+  private static final SqlAbstractParserImpl.Metadata SQL_PARSER_METADATA = SqlParser.create("",
+      SqlParser.configBuilder().setConformance(SqlConformanceEnum.BABEL).setParserFactory(SqlBabelParserImpl.FACTORY)
+          .build()).getMetadata();
+
   /**
-   * 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
+   * Validates the following:
+   * 1) Checks if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
    */
   public static boolean validate(Schema schema) {
-    return validate(schema, LOGGER);
+    return validate(schema, true, 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 if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
+   *
+   * @param validateFieldNames if false, does not validate field names. This is to prevent validation failing on existing schemas with invalid field names during a schema update
    */
-  public static boolean validate(Schema schema, Logger logger) {
+  public static boolean validate(Schema schema, boolean validateFieldNames, @Nullable 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;
-              }
+          if (validateFieldNames && !isValidFieldName(fieldSpec)) {
+            return false;
+          }
+          if (!isValidTransformFunction(fieldSpec)) {
+            return false;
+          }
+          FieldSpec.FieldType fieldType = fieldSpec.getFieldType();
+          if (fieldType.equals(FieldSpec.FieldType.DATE_TIME)) {
+            if (!isValidDateTimeFieldSpec(fieldSpec)) {
+              return false;
             }
-          } else if (fieldSpec.getFieldType().equals(FieldSpec.FieldType.TIME)) {
-            TimeFieldSpec timeFieldSpec = (TimeFieldSpec) fieldSpec;
-            TimeGranularitySpec incomingGranularitySpec = timeFieldSpec.getIncomingGranularitySpec();
-            TimeGranularitySpec outgoingGranularitySpec = timeFieldSpec.getOutgoingGranularitySpec();
-
-            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;
-                }
-              }
+          } else if (fieldType.equals(FieldSpec.FieldType.TIME)) {
+            if (!isValidTimeFieldSpec(fieldSpec)) {
+              return false;
             }
           }
         }
       }
     } catch (Exception e) {
-      logger.error("Exception in validating schema {}", schema.getSchemaName(), e);
+      LOGGER.error("Exception in validating schema {}", schema.getSchemaName(), e);
       return false;
     }
     return schema.validate(logger);
   }
+
+  /**
+   * Checks if any of the keywords which are reserved under the sql parser are used
+   */
+  private static boolean isValidFieldName(FieldSpec fieldSpec) {
+    String columnName = fieldSpec.getName();
+    if (SQL_PARSER_METADATA.isReservedWord(columnName.toUpperCase())) {
+      LOGGER.error("Cannot use SQL reserved word {} as field name in the schema", columnName);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Checks for valid transform function string
+   */
+  private static boolean isValidTransformFunction(FieldSpec fieldSpec) {
+    String column = fieldSpec.getName();
+    String transformFunction = fieldSpec.getTransformFunction();
+    if (transformFunction != null) {
+      try {
+        List<String> arguments = FunctionEvaluatorFactory.getExpressionEvaluator(fieldSpec).getArguments();

Review comment:
       Useful to check for circular dependency here? What if we have column A used as argument for column B and vice versa? Should we not resolve all arguments of all fields before declaring a bad schema?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,134 @@
 
   private static final Logger LOGGER = LoggerFactory.getLogger(SchemaUtils.class);
 
+  private static final SqlAbstractParserImpl.Metadata SQL_PARSER_METADATA = SqlParser.create("",
+      SqlParser.configBuilder().setConformance(SqlConformanceEnum.BABEL).setParserFactory(SqlBabelParserImpl.FACTORY)
+          .build()).getMetadata();
+
   /**
-   * 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
+   * Validates the following:
+   * 1) Checks if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
    */
   public static boolean validate(Schema schema) {
-    return validate(schema, LOGGER);
+    return validate(schema, true, 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 if sql reserved keywords are being used as field names. This check can be disabled, for not breaking existing schemas with invalid names
+   * 2) 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}
+   * 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(Logger)}
+   *
+   * @param validateFieldNames if false, does not validate field names. This is to prevent validation failing on existing schemas with invalid field names during a schema update
    */
-  public static boolean validate(Schema schema, Logger logger) {
+  public static boolean validate(Schema schema, boolean validateFieldNames, @Nullable 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;
-              }
+          if (validateFieldNames && !isValidFieldName(fieldSpec)) {
+            return false;
+          }
+          if (!isValidTransformFunction(fieldSpec)) {
+            return false;
+          }
+          FieldSpec.FieldType fieldType = fieldSpec.getFieldType();
+          if (fieldType.equals(FieldSpec.FieldType.DATE_TIME)) {
+            if (!isValidDateTimeFieldSpec(fieldSpec)) {
+              return false;
             }
-          } else if (fieldSpec.getFieldType().equals(FieldSpec.FieldType.TIME)) {
-            TimeFieldSpec timeFieldSpec = (TimeFieldSpec) fieldSpec;
-            TimeGranularitySpec incomingGranularitySpec = timeFieldSpec.getIncomingGranularitySpec();
-            TimeGranularitySpec outgoingGranularitySpec = timeFieldSpec.getOutgoingGranularitySpec();
-
-            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;
-                }
-              }
+          } else if (fieldType.equals(FieldSpec.FieldType.TIME)) {
+            if (!isValidTimeFieldSpec(fieldSpec)) {
+              return false;
             }
           }
         }
       }
     } catch (Exception e) {
-      logger.error("Exception in validating schema {}", schema.getSchemaName(), e);
+      LOGGER.error("Exception in validating schema {}", schema.getSchemaName(), e);
       return false;
     }
     return schema.validate(logger);
   }
+
+  /**
+   * Checks if any of the keywords which are reserved under the sql parser are used
+   */
+  private static boolean isValidFieldName(FieldSpec fieldSpec) {
+    String columnName = fieldSpec.getName();
+    if (SQL_PARSER_METADATA.isReservedWord(columnName.toUpperCase())) {
+      LOGGER.error("Cannot use SQL reserved word {} as field name in the schema", columnName);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Checks for valid transform function string
+   */
+  private static boolean isValidTransformFunction(FieldSpec fieldSpec) {
+    String column = fieldSpec.getName();
+    String transformFunction = fieldSpec.getTransformFunction();
+    if (transformFunction != null) {
+      try {
+        List<String> arguments = FunctionEvaluatorFactory.getExpressionEvaluator(fieldSpec).getArguments();
+        // output column used as input
+        if (arguments.contains(column)) {
+          LOGGER.error("The arguments of transform function: {}, should not contain the destination column: {}",

Review comment:
       Let us be consistent -- call one of "field" or "column" in error messages. I am fine with either one, but consistency is useful.




----------------------------------------------------------------
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.

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



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