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/06 19:20:29 UTC

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

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



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,130 @@
 
   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(Locale.ROOT))) {

Review comment:
       Why using `Locale.ROOT`? Seems we are using the default (`columnName.toUpperCase()`) everywhere else

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/manager/realtime/RealtimeTableDataManager.java
##########
@@ -354,7 +354,7 @@ private boolean isValid(Schema schema, IndexingConfig indexingConfig) {
       }
     }
     // 2. We want to get the schema errors, if any, even if isValid is false;
-    if (!SchemaUtils.validate(schema, _logger)) {
+    if (!SchemaUtils.validate(schema, false, _logger)) {

Review comment:
       Why skipping the fields validation?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,130 @@
 
   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(Locale.ROOT))) {
+      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) {
+      FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(fieldSpec);

Review comment:
       You may want try-catch around this to prevent throwing exception for the validation

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/SchemaUtils.java
##########
@@ -39,65 +48,130 @@
 
   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(Locale.ROOT))) {
+      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) {
+      FunctionEvaluator functionEvaluator = FunctionEvaluatorFactory.getExpressionEvaluator(fieldSpec);
+      if (functionEvaluator != null) {

Review comment:
       `functionEvaluator` should not be `null`




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