[ 
https://issues.apache.org/jira/browse/BEAM-9035?focusedWorklogId=389533&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-389533
 ]

ASF GitHub Bot logged work on BEAM-9035:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 19/Feb/20 17:54
            Start Date: 19/Feb/20 17:54
    Worklog Time Spent: 10m 
      Work Description: TheNeuralBit commented on pull request #10413: 
[BEAM-9035] Typed options for Row Schema and Field
URL: https://github.com/apache/beam/pull/10413#discussion_r381443180
 
 

 ##########
 File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
 ##########
 @@ -950,6 +1007,338 @@ public int hashCode() {
     }
   }
 
+  public static class Options implements Serializable {
+    private Map<String, Option> options;
+
+    @Override
+    public String toString() {
+      TreeMap sorted = new TreeMap(options);
+      return "{" + sorted + '}';
+    }
+
+    Map<String, Option> getAllOptions() {
+      return options;
+    }
+
+    public Set<String> getOptionNames() {
+      return options.keySet();
+    }
+
+    public boolean hasOptions() {
+      return options.size() > 0;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      Options options1 = (Options) o;
+      if (!options.keySet().equals(options1.options.keySet())) {
+        return false;
+      }
+      for (Map.Entry<String, Option> optionEntry : options.entrySet()) {
+        Option thisOption = optionEntry.getValue();
+        Option otherOption = options1.options.get(optionEntry.getKey());
+        if (!thisOption.getType().equals(otherOption.getType())) {
+          return false;
+        }
+        switch (thisOption.getType().getTypeName()) {
+          case BYTE:
+          case INT16:
+          case INT32:
+          case INT64:
+          case DECIMAL:
+          case FLOAT:
+          case DOUBLE:
+          case STRING:
+          case DATETIME:
+          case BOOLEAN:
+          case ARRAY:
+          case ITERABLE:
+          case MAP:
+          case ROW:
+          case LOGICAL_TYPE:
+            if (!thisOption.getValue().equals(otherOption.getValue())) {
+              return false;
+            }
+            break;
+          case BYTES:
+            if (!Arrays.equals((byte[]) thisOption.getValue(), 
otherOption.getValue())) {
+              return false;
+            }
+        }
+      }
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(options);
+    }
+
+    static class Option implements Serializable {
+      Option(FieldType type, Object value) {
+        this.type = type;
+        this.value = value;
+      }
+
+      private FieldType type;
+      private Object value;
+
+      @SuppressWarnings("TypeParameterUnusedInFormals")
+      <T> T getValue() {
+        return (T) value;
+      }
+
+      FieldType getType() {
+        return type;
+      }
+
+      @Override
+      public String toString() {
+        return "Option{type=" + type + ", value=" + value + '}';
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) {
+          return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+          return false;
+        }
+        Option option = (Option) o;
+        return Objects.equals(type, option.type) && Objects.equals(value, 
option.value);
+      }
+
+      @Override
+      public int hashCode() {
+        return Objects.hash(type, value);
+      }
+    }
+
+    public static class Builder {
+      private Map<String, Option> options;
+
+      Builder(Map<String, Option> init) {
+        this.options = new HashMap<>(init);
+      }
+
+      Builder() {
+        this(new HashMap<>());
+      }
+
+      public Builder setByteOption(String optionName, Byte value) {
+        setOption(optionName, FieldType.BYTE, value);
+        return this;
+      }
+
+      public Builder setBytesOption(String optionName, byte[] value) {
+        setOption(optionName, FieldType.BYTES, value);
+        return this;
+      }
+
+      public Builder setInt16Option(String optionName, Short value) {
+        setOption(optionName, FieldType.INT16, value);
+        return this;
+      }
+
+      public Builder setInt32Option(String optionName, Integer value) {
+        setOption(optionName, FieldType.INT32, value);
+        return this;
+      }
+
+      public Builder setInt64Option(String optionName, Long value) {
+        setOption(optionName, FieldType.INT64, value);
+        return this;
+      }
+
+      public Builder setDecimalOption(String optionName, BigDecimal value) {
+        setOption(optionName, FieldType.DECIMAL, value);
+        return this;
+      }
+
+      public Builder setFloatOption(String optionName, Float value) {
+        setOption(optionName, FieldType.FLOAT, value);
+        return this;
+      }
+
+      public Builder setDoubleOption(String optionName, Double value) {
+        setOption(optionName, FieldType.DOUBLE, value);
+        return this;
+      }
+
+      public Builder setStringOption(String optionName, String value) {
+        setOption(optionName, FieldType.STRING, value);
+        return this;
+      }
+
+      public Builder setDateTimeOption(String optionName, ReadableDateTime 
value) {
+        setOption(optionName, FieldType.DATETIME, value);
+        return this;
+      }
+
+      public Builder setBooleanOption(String optionName, Boolean value) {
+        setOption(optionName, FieldType.BOOLEAN, value);
+        return this;
+      }
+
+      public <V> Builder setArrayOption(String optionName, FieldType 
arrayType, List<V> value) {
+        setOption(optionName, arrayType, value);
+        return this;
+      }
+
+      public <K, V> Builder setMapOption(String optionName, FieldType mapType, 
Map<K, V> value) {
+        setOption(optionName, mapType, value);
+        return this;
+      }
+
+      public Builder setRowOption(String optionName, Row value) {
+        setOption(optionName, FieldType.row(value.getSchema()), value);
+        return this;
+      }
+
+      public Builder setOption(String optionName, FieldType fieldType, Object 
value) {
+        if (value == null) {
+          removeOption(optionName);
+        } else {
+          options.put(
+              optionName, new Option(fieldType, ValueUtils.verify(value, 
fieldType, optionName)));
 
 Review comment:
   Is it really that unnatural? My understanding (based on my quick look at the 
[docs](https://developers.google.com/protocol-buffers/docs/proto#customoptions))
 is that protobuf options are themselves described by protobuf message 
descriptors. So they have a structure that we are already mapping to a nested 
Beam schema in another context. If we just used the protobuf schema provider to 
generate a schema from the option's descriptor(s), then any dot in the name 
would be referencing a level of nesting, not a literal character in the field 
name, wouldn't it?
   
   I see your point about URNs, they usually define some namespaces separated 
by `:` characters, is this a slippery slope and we need to nest them as well? 
But that wouldn't be necessary since we don't have any issue with `:` in field 
names (that I'm aware of).
   
   As far as code complexity - my hope was that this would actually reduce code 
complexity, since we can re-use a lot of existing infrastructure: 
RowWithStorage, Schema, FieldAccessDescriptor, and the protobuf SchemaProvider.
   
   To be clear, I'm ok with keeping the options as-is. I just thought this may 
be a way to re-use more code. But you know the problem you're trying to solve 
better than I do so I'll defer to you if you think this doesn't make sense.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 389533)
    Time Spent: 3.5h  (was: 3h 20m)

> Typed options for Row Schema and Fields
> ---------------------------------------
>
>                 Key: BEAM-9035
>                 URL: https://issues.apache.org/jira/browse/BEAM-9035
>             Project: Beam
>          Issue Type: Sub-task
>          Components: sdk-java-core
>            Reporter: Alex Van Boxel
>            Assignee: Alex Van Boxel
>            Priority: Major
>             Fix For: 2.19.0
>
>          Time Spent: 3.5h
>  Remaining Estimate: 0h
>
> This is the first issue of a multipart commit: this ticket implements the 
> basic infrastructure of options on row and field.
> Full explanation:
> Introduce the concept of Options in Beam Schema’s to add extra context to 
> fields and schema. In contracts to metadata, options would be added to 
> fields, logical types and rows. In the options schema convertors can add 
> options/annotations/decorators that were in the original schema, this context 
> can be used in the rest of the pipeline for specific transformations or 
> augment the end schema in the target output.
> Examples of options are:
>  * informational: like the source of the data, ...
>  * drive decisions further in the pipeline: flatten a row into another, 
> rename a field, ...
>  * influence something in the output: like cluster index, primary key, ...
>  * logical type information
> And option is a key/typed value combination. The advantages of having the 
> value types is: 
>  * Having strongly typed options would give a *portable way of Logical Types* 
> to have structured information that could be shared over different languages.
>  * This could keep the type intact when mapping from a formats that have 
> strongly typed options (example: Protobuf).
> This is part of a multi ticket implementation. The following tickets are 
> related:
>  # Typed options for Row Schema and Fields
>  # Convert Proto Options to Beam Schema options
>  # Convert Avro extra information for Beam string options
>  # Replace meta data with Logical Type options
>  # Extract meta data in Calcite SQL to Beam options
>  # Extract meta data in Zeta SQL to Beam options
>  # Add java example of using option in a transform 
> This feature is discussed with Reuven Lax, Brian Hulette



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to