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 2021/03/05 05:39:36 UTC

[GitHub] [incubator-pinot] icefury71 commented on a change in pull request #6485: [WIP] add schema validation mechanism for table config

icefury71 commented on a change in pull request #6485:
URL: https://github.com/apache/incubator-pinot/pull/6485#discussion_r588042899



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/validator/PinotConfigValidator.java
##########
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util.validator;
+
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import java.io.Serializable;
+import java.util.List;
+
+
+public interface PinotConfigValidator extends Serializable {

Review comment:
       javadoc please

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/validator/BasePinotConfigValidator.java
##########
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util.validator;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.github.fge.jackson.JsonLoader;
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import com.github.fge.jsonschema.core.report.LogLevel;
+import com.github.fge.jsonschema.core.report.ProcessingMessage;
+import com.github.fge.jsonschema.core.report.ProcessingReport;
+import com.github.fge.jsonschema.main.JsonSchema;
+import com.github.fge.jsonschema.main.JsonSchemaFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.pinot.spi.utils.JsonUtils;
+
+/**
+* Base implementation for validating JSON schemas
+* */
+abstract public class BasePinotConfigValidator implements PinotConfigValidator {
+
+  protected JsonSchema _schema;
+  protected ProcessingReport _report;
+
+  /**
+   * Load schema from project resources
+   * */
+  protected void loadSchema(String path)
+      throws IOException, ProcessingException {
+    JsonNode node = JsonLoader.fromResource(path);
+    JsonSchemaFactory factory = JsonSchemaFactory.byDefault();

Review comment:
       Does this need to be inside the method or can be a class member variable ?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/validator/BasePinotConfigValidator.java
##########
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util.validator;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.github.fge.jackson.JsonLoader;
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import com.github.fge.jsonschema.core.report.LogLevel;
+import com.github.fge.jsonschema.core.report.ProcessingMessage;
+import com.github.fge.jsonschema.core.report.ProcessingReport;
+import com.github.fge.jsonschema.main.JsonSchema;
+import com.github.fge.jsonschema.main.JsonSchemaFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.pinot.spi.utils.JsonUtils;
+
+/**
+* Base implementation for validating JSON schemas
+* */
+abstract public class BasePinotConfigValidator implements PinotConfigValidator {
+
+  protected JsonSchema _schema;
+  protected ProcessingReport _report;
+
+  /**
+   * Load schema from project resources
+   * */
+  protected void loadSchema(String path)
+      throws IOException, ProcessingException {
+    JsonNode node = JsonLoader.fromResource(path);
+    JsonSchemaFactory factory = JsonSchemaFactory.byDefault();
+    _schema = factory.getJsonSchema(node);
+  }
+
+  /**
+  * Validate passed jsonStr schema against the loaded schema from resources
+  * */
+  @Override
+  public boolean validate(String jsonStr)
+      throws ProcessingException {
+    JsonNode node = null;
+    try {
+      node = JsonUtils.stringToJsonNode(jsonStr);
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    _report = _schema.validate(node);
+    return _report.isSuccess();
+  }
+
+  /**
+   * Report validation messages if validation fail
+   * */
+  @Override
+  public List<String> getValidationMessages() {

Review comment:
       This method assumes the caller has already invoked validate(jsonStr) : see my comment below in the interface class regarding simplifying this dependency

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/validator/PinotConfigValidator.java
##########
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util.validator;
+
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import java.io.Serializable;
+import java.util.List;
+
+
+public interface PinotConfigValidator extends Serializable {
+
+  boolean validate(String jsonStr)
+      throws ProcessingException;
+
+  List<String> getValidationMessages();

Review comment:
       I wonder if we should have just one method - that takes a jsonStr and returns a list of validation messages. That way there's no dependency on the caller side to invoke validate first and then invoke getValidationMessages. 

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/validator/PinotConfigSchemasValidator.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util.validator;
+
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import java.io.IOException;
+
+/**
+ * Factory to help create validator for [Table Config,]
+ * */
+public class PinotConfigSchemasValidator extends BasePinotConfigValidator {

Review comment:
       Rename to PinotTableConfigSchemaValidator ?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/validator/BasePinotConfigValidator.java
##########
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util.validator;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.github.fge.jackson.JsonLoader;
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import com.github.fge.jsonschema.core.report.LogLevel;
+import com.github.fge.jsonschema.core.report.ProcessingMessage;
+import com.github.fge.jsonschema.core.report.ProcessingReport;
+import com.github.fge.jsonschema.main.JsonSchema;
+import com.github.fge.jsonschema.main.JsonSchemaFactory;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.pinot.spi.utils.JsonUtils;
+
+/**
+* Base implementation for validating JSON schemas
+* */
+abstract public class BasePinotConfigValidator implements PinotConfigValidator {
+
+  protected JsonSchema _schema;
+  protected ProcessingReport _report;
+
+  /**
+   * Load schema from project resources
+   * */
+  protected void loadSchema(String path)
+      throws IOException, ProcessingException {
+    JsonNode node = JsonLoader.fromResource(path);
+    JsonSchemaFactory factory = JsonSchemaFactory.byDefault();
+    _schema = factory.getJsonSchema(node);
+  }
+
+  /**
+  * Validate passed jsonStr schema against the loaded schema from resources
+  * */
+  @Override
+  public boolean validate(String jsonStr)
+      throws ProcessingException {
+    JsonNode node = null;
+    try {
+      node = JsonUtils.stringToJsonNode(jsonStr);
+    } catch (IOException e) {
+      e.printStackTrace();

Review comment:
       Should we throw if we can't get a valid Json ? 

##########
File path: pinot-core/src/test/java/org/apache/pinot/util/validator/PinotConfigSchemasValidatorTest.java
##########
@@ -0,0 +1,9 @@
+package org.apache.pinot.util.validator;
+
+import org.testng.annotations.Test;
+
+
+public class PinotConfigSchemasValidatorTest {

Review comment:
       Empty ?

##########
File path: pinot-core/src/test/java/org/apache/pinot/util/validator/BasePinotConfigValidatorTest.java
##########
@@ -0,0 +1,62 @@
+package org.apache.pinot.util.validator;
+
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import org.apache.pinot.core.util.validator.BasePinotConfigValidator;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class BasePinotConfigValidatorTest extends BasePinotConfigValidator {
+
+  @Test
+  public void testLoadingSchema()
+      throws IOException, ProcessingException {
+    // _schema object is null currently
+    Assert.assertNull(_schema);
+
+    loadSchema("/schemas/tableConfig.json");
+    // _schema is store a valid schema after loading
+    Assert.assertNotNull(_schema);
+
+    // fail and throw IOException if schema not found
+    try {
+      loadSchema("/no/schema/here");
+      Assert.fail("schema not found");
+    } catch (IOException e) {
+      // expected
+    }
+  }
+
+  @Test
+  public void testValidateSchema()
+      throws IOException, ProcessingException {
+
+    // given a valid schema, validate should returns true with not error messages
+    String airlinesTableConfig = readSampleJson("/schemas/good_airlines_table_config.json");
+    loadSchema("/schemas/tableConfig.json");
+    boolean result = validate(airlinesTableConfig);
+    Assert.assertEquals(getValidationMessages().size(), 0);
+    Assert.assertTrue(result);
+
+    // given a invalid schema [invalid key is table_type], validate should returns false
+    airlinesTableConfig = readSampleJson("/schemas/bad_airlines_table_config.json");
+    loadSchema("/schemas/tableConfig.json");
+    result = validate(airlinesTableConfig);
+    Assert.assertEquals(getValidationMessages().size(), 1);
+    Assert.assertFalse(result);
+  }
+
+  private String readSampleJson(String path)

Review comment:
       Do we need this ? I feel some Json Utils method might do the trick here ?

##########
File path: pinot-core/src/test/java/org/apache/pinot/util/validator/BasePinotConfigValidatorTest.java
##########
@@ -0,0 +1,62 @@
+package org.apache.pinot.util.validator;
+
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import org.apache.pinot.core.util.validator.BasePinotConfigValidator;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+
+public class BasePinotConfigValidatorTest extends BasePinotConfigValidator {
+
+  @Test
+  public void testLoadingSchema()
+      throws IOException, ProcessingException {
+    // _schema object is null currently
+    Assert.assertNull(_schema);
+
+    loadSchema("/schemas/tableConfig.json");
+    // _schema is store a valid schema after loading
+    Assert.assertNotNull(_schema);
+
+    // fail and throw IOException if schema not found
+    try {
+      loadSchema("/no/schema/here");
+      Assert.fail("schema not found");

Review comment:
       nit: rename error message to : "should fail for missing schema"

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/util/validator/PinotConfigSchemasValidator.java
##########
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.core.util.validator;
+
+import com.github.fge.jsonschema.core.exceptions.ProcessingException;
+import java.io.IOException;
+
+/**
+ * Factory to help create validator for [Table Config,]
+ * */
+public class PinotConfigSchemasValidator extends BasePinotConfigValidator {
+
+
+  private static final String TABLE_CONFIG_SCHEMA = "/schemas/tableConfig.json";
+
+  private PinotConfigSchemasValidator(String schema)
+      throws IOException, ProcessingException {
+    loadSchema(schema);
+  }
+
+  /**
+   * Create validator for table config
+   * */
+  public static PinotConfigSchemasValidator forTableConfig()

Review comment:
       I think we should make this a singleton.




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