You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by sa...@apache.org on 2023/05/02 09:54:20 UTC

[pinot] branch master updated: API to expose the contract/rules imposed by pinot on tableConfig (#10655)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f5d75866f6 API to expose the contract/rules imposed by pinot on tableConfig (#10655)
f5d75866f6 is described below

commit f5d75866f6d833504ca5e0675066cec0ea5dab0a
Author: Shounak kulkarni <sh...@gmail.com>
AuthorDate: Tue May 2 15:24:13 2023 +0530

    API to expose the contract/rules imposed by pinot on tableConfig (#10655)
    
    * Wrapper to consolidate the metadata for FieldSpec
    
    It consists of all the possible DataTypes for each FieldType and the default null value for each of these combinations.
    
    * API endpoint to expose the FieldSpecMetadata
    
    * decouple fieldType-dataType validation from schema validation
    
    * checkstyle fixes
    
    * endpoint rename
    
    * assign FIELD_SPEC_METADATA directly from a static block
    
    * cleanup
    
    * changed the endpoint to /schemas/fieldSpec
    
    * added data types info to FieldSpecMetadata
---
 .../api/resources/PinotSchemaRestletResource.java  |  17 ++++
 .../java/org/apache/pinot/spi/data/FieldSpec.java  |  82 +++++++++++++++++
 .../java/org/apache/pinot/spi/data/Schema.java     | 100 +++++++++++----------
 3 files changed, 153 insertions(+), 46 deletions(-)

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 cbc25f5ab9..3d665ac9ca 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
@@ -68,6 +68,7 @@ import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
 import org.apache.pinot.core.auth.ManualAuthorization;
 import org.apache.pinot.segment.local.utils.SchemaUtils;
 import org.apache.pinot.spi.config.table.TableConfig;
+import org.apache.pinot.spi.data.FieldSpec;
 import org.apache.pinot.spi.data.Schema;
 import org.apache.pinot.spi.utils.JsonUtils;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
@@ -331,6 +332,22 @@ public class PinotSchemaRestletResource {
     }
   }
 
+  /**
+   * Gets the metadata on the valid {@link org.apache.pinot.spi.data.FieldSpec.DataType} for each
+   * {@link org.apache.pinot.spi.data.FieldSpec.FieldType} and the default null values for each combination
+   */
+  @GET
+  @Produces(MediaType.APPLICATION_JSON)
+  @Path("/schemas/fieldSpec")
+  @ApiOperation(value = "Get fieldSpec metadata", notes = "Get fieldSpec metadata")
+  public String getFieldSpecMetadata() {
+    try {
+      return JsonUtils.objectToString(FieldSpec.FIELD_SPEC_METADATA);
+    } catch (Exception e) {
+      throw new ControllerApplicationException(LOGGER, e.getMessage(), Response.Status.INTERNAL_SERVER_ERROR, e);
+    }
+  }
+
   private void validateSchemaName(String schemaName) {
     if (StringUtils.isBlank(schemaName)) {
       throw new ControllerApplicationException(LOGGER, "Invalid schema. Reason: 'schemaName' should not be null",
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java b/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
index 32834570ca..3c931204a9 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/data/FieldSpec.java
@@ -19,10 +19,13 @@
 package org.apache.pinot.spi.data;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.databind.node.ObjectNode;
 import java.io.Serializable;
 import java.math.BigDecimal;
 import java.sql.Timestamp;
+import java.util.HashMap;
+import java.util.Map;
 import javax.annotation.Nullable;
 import org.apache.pinot.spi.utils.BooleanUtils;
 import org.apache.pinot.spi.utils.BytesUtils;
@@ -64,6 +67,34 @@ public abstract class FieldSpec implements Comparable<FieldSpec>, Serializable {
   public static final BigDecimal DEFAULT_METRIC_NULL_VALUE_OF_BIG_DECIMAL = BigDecimal.ZERO;
   public static final String DEFAULT_METRIC_NULL_VALUE_OF_STRING = "null";
   public static final byte[] DEFAULT_METRIC_NULL_VALUE_OF_BYTES = new byte[0];
+  public static final FieldSpecMetadata FIELD_SPEC_METADATA;
+
+  static {
+    // The metadata on the valid list of {@link DataType} for each {@link FieldType}
+    // and the default null values for each combination
+    FIELD_SPEC_METADATA = new FieldSpecMetadata();
+    for (FieldType fieldType : FieldType.values()) {
+      FieldTypeMetadata fieldTypeMetadata = new FieldTypeMetadata();
+      for (DataType dataType : DataType.values()) {
+        try {
+          Schema.validate(fieldType, dataType);
+          try {
+            fieldTypeMetadata.put(dataType, new DataTypeMetadata(getDefaultNullValue(fieldType, dataType, null)));
+          } catch (IllegalStateException ignored) {
+            // default null value not defined for the (DataType, FieldType) combination
+            // defaulting to null in such cases
+            fieldTypeMetadata.put(dataType, new DataTypeMetadata(null));
+          }
+        } catch (IllegalStateException ignored) {
+          // invalid DataType for the given FieldType
+        }
+      }
+      FIELD_SPEC_METADATA.put(fieldType, fieldTypeMetadata);
+    }
+    for (DataType dataType : DataType.values()) {
+      FIELD_SPEC_METADATA.put(dataType, new DataTypeProperties(dataType));
+    }
+  }
 
   protected String _name;
   protected DataType _dataType;
@@ -563,4 +594,55 @@ public abstract class FieldSpec implements Comparable<FieldSpec>, Serializable {
             && EqualityUtils.isEqual(_dataType, oldFieldSpec._dataType)
             && EqualityUtils.isEqual(_isSingleValueField, oldFieldSpec._isSingleValueField);
   }
+
+  public static class FieldSpecMetadata {
+    @JsonProperty("fieldTypes")
+    public Map<FieldType, FieldTypeMetadata> _fieldTypes = new HashMap<>();
+    @JsonProperty("dataTypes")
+    public Map<DataType, DataTypeProperties> _dataTypes = new HashMap<>();
+
+    void put(FieldType type, FieldTypeMetadata metadata) {
+      _fieldTypes.put(type, metadata);
+    }
+
+    void put(DataType type, DataTypeProperties metadata) {
+      _dataTypes.put(type, metadata);
+    }
+  }
+
+  public static class FieldTypeMetadata {
+    @JsonProperty("allowedDataTypes")
+    public Map<DataType, DataTypeMetadata> _allowedDataTypes = new HashMap<>();
+
+    void put(DataType dataType, DataTypeMetadata metadata) {
+      _allowedDataTypes.put(dataType, metadata);
+    }
+  }
+
+  public static class DataTypeMetadata {
+    @JsonProperty("nullDefault")
+    public Object _nullDefault;
+
+    public DataTypeMetadata(Object nullDefault) {
+      _nullDefault = nullDefault;
+    }
+  }
+
+  public static class DataTypeProperties {
+    @JsonProperty("storedType")
+    public final DataType _storedType;
+    @JsonProperty("size")
+    public final int _size;
+    @JsonProperty("sortable")
+    public final boolean _sortable;
+    @JsonProperty("numeric")
+    public final boolean _numeric;
+
+    public DataTypeProperties(DataType dataType) {
+      _storedType = dataType._storedType;
+      _sortable = dataType._sortable;
+      _numeric = dataType._numeric;
+      _size = dataType._size;
+    }
+  }
 }
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 b5a3b11c0b..4bd63a16df 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
@@ -104,6 +104,56 @@ public final class Schema implements Serializable {
     return JsonUtils.inputStreamToObject(schemaInputStream, Schema.class);
   }
 
+  public static void validate(FieldType fieldType, DataType dataType) {
+    switch (fieldType) {
+      case DIMENSION:
+      case TIME:
+      case DATE_TIME:
+        switch (dataType) {
+          case INT:
+          case LONG:
+          case FLOAT:
+          case DOUBLE:
+          case BIG_DECIMAL:
+          case BOOLEAN:
+          case TIMESTAMP:
+          case STRING:
+          case JSON:
+          case BYTES:
+            break;
+          default:
+            throw new IllegalStateException(
+                "Unsupported data type: " + dataType + " in DIMENSION/TIME field");
+        }
+        break;
+      case METRIC:
+        switch (dataType) {
+          case INT:
+          case LONG:
+          case FLOAT:
+          case DOUBLE:
+          case BIG_DECIMAL:
+          case BYTES:
+            break;
+          default:
+            throw new IllegalStateException("Unsupported data type: " + dataType + " in METRIC field");
+        }
+        break;
+      case COMPLEX:
+        switch (dataType) {
+          case STRUCT:
+          case MAP:
+          case LIST:
+            break;
+          default:
+            throw new IllegalStateException("Unsupported data type: " + dataType + " in COMPLEX field");
+        }
+        break;
+      default:
+        throw new IllegalStateException("Unsupported data type: " + dataType + " for field");
+    }
+  }
+
   /**
    * NOTE: schema name could be null in tests
    */
@@ -450,52 +500,10 @@ public final class Schema implements Serializable {
       FieldType fieldType = fieldSpec.getFieldType();
       DataType dataType = fieldSpec.getDataType();
       String fieldName = fieldSpec.getName();
-      switch (fieldType) {
-        case DIMENSION:
-        case TIME:
-        case DATE_TIME:
-          switch (dataType) {
-            case INT:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
-            case BIG_DECIMAL:
-            case BOOLEAN:
-            case TIMESTAMP:
-            case STRING:
-            case JSON:
-            case BYTES:
-              break;
-            default:
-              throw new IllegalStateException(
-                  "Unsupported data type: " + dataType + " in DIMENSION/TIME field: " + fieldName);
-          }
-          break;
-        case METRIC:
-          switch (dataType) {
-            case INT:
-            case LONG:
-            case FLOAT:
-            case DOUBLE:
-            case BIG_DECIMAL:
-            case BYTES:
-              break;
-            default:
-              throw new IllegalStateException("Unsupported data type: " + dataType + " in METRIC field: " + fieldName);
-          }
-          break;
-        case COMPLEX:
-          switch (dataType) {
-            case STRUCT:
-            case MAP:
-            case LIST:
-              break;
-            default:
-              throw new IllegalStateException("Unsupported data type: " + dataType + " in COMPLEX field: " + fieldName);
-          }
-          break;
-        default:
-          throw new IllegalStateException("Unsupported data type: " + dataType + " for field: " + fieldName);
+      try {
+        validate(fieldType, dataType);
+      } catch (IllegalStateException e) {
+        throw new IllegalStateException(e.getMessage() + ": " + fieldName);
       }
     }
   }


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