TheNeuralBit commented on a change in pull request #10767: Document Beam Schemas
URL: https://github.com/apache/beam/pull/10767#discussion_r408491512
 
 

 ##########
 File path: website/src/documentation/programming-guide.md
 ##########
 @@ -1970,7 +1976,1078 @@ records.apply("WriteToText",
 See the [Beam-provided I/O Transforms]({{site.baseurl 
}}/documentation/io/built-in/)
 page for a list of the currently available I/O transforms.
 
-## 6. Data encoding and type safety {#data-encoding-and-type-safety}
+## 6. Schemas {#schemas}
+Often, the type of records being processed have an obvious structure. Common 
Beam sources produce
+JSON, Avro, Protocol Buffer, or database row objects; all of these types have 
well defined structures, 
+structures that can often be determined by examining the type. Even within a 
SDK pipeline, Simple Java POJOs 
+(or  equivalent structures in other languages) are often used as intermediate 
types, and these also have a
+ clear structure that can be inferred by inspecting the class. By 
understanding the structure of a pipeline’s 
+ records, we can provide much more concise APIs for data processing.
+ 
+### 6.1. What is a schema {#what-is-a-schema}
+Most structured records share some common characteristics: 
+* They can be subdivided into separate named fields. Fields usually have 
string names, but sometimes - as in the case of indexed
+ tuples - have numerical indices instead.
+* There is a confined list of primitive types that a field can have. These 
often match primitive types in most programming 
+ languages: int, long, string, etc.
+* Often a field type can be marked as optional (sometimes referred to as 
nullable) or required.
+
+In addition, often records have a nested structure. A nested structure occurs 
when a field itself has subfields so the 
+type of the field itself has a schema. Fields that are  array or map types is 
also a common feature of these structured 
+records.
+
+For example, consider the following schema, representing actions in a 
fictitious e-commerce company:
+
+**Purchase**
+<table>
+  <thead>
+    <tr class="header">
+      <th><b>Field Name</b></th>
+      <th><b>Field Type</b></th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>userId</td>
+      <td>STRING</td>      
+    </tr>
+    <tr>
+      <td>itemId</td>
+      <td>INT64</td>      
+    </tr>
+    <tr>
+      <td>shippingAddress</td>
+      <td>ROW(ShippingAddress)</td>      
+    </tr>
+    <tr>
+      <td>cost</td>
+      <td>INT64</td>      
+    </tr>
+    <tr>
+      <td>transactions</td>
+      <td>ARRAY[ROW(Transaction)]</td>      
+    </tr>                  
+  </tbody>
+</table>
+<br/>
+
+**ShippingAddress**
+<table>
+  <thead>
+    <tr class="header">
+      <th><b>Field Name</b></th>
+      <th><b>Field Type</b></th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>streetAddress</td>
+      <td>STRING</td>      
+    </tr>
+    <tr>
+      <td>city</td>
+      <td>STRING</td>      
+    </tr>
+    <tr>
+      <td>state</td>
+      <td>nullable STRING</td>      
+    </tr>
+    <tr>
+      <td>country</td>
+      <td>STRING</td>      
+    </tr>
+    <tr>
+      <td>postCode</td>
+      <td>STRING</td>      
+    </tr>                  
+  </tbody>
+</table> 
+<br/>
+
+**Transaction**
+<table>
+  <thead>
+    <tr class="header">
+      <th><b>Field Name</b></th>
+      <th><b>Field Type</b></th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>bank</td>
+      <td>STRING</td>      
+    </tr>
+    <tr>
+      <td>purchaseAmount</td>
+      <td>DOUBLE</td>      
+    </tr>                  
+  </tbody>
+</table>
+<br/>
+
+Purchase event records are represented by the above purchase schema. Each 
purchase event contains a shipping address, which
+is a nested row containing its own schema. Each purchase also contains a list 
of credit-card transactions 
+(a list, because a purchase might be split across multiple credit cards); each 
item in the transaction list is a row 
+with its own schema.
+
+This provides an abstract description of the types involved, one that is 
abstracted away from any specific programming 
+language.
+
+Schemas provide us a type-system for Beam records that is independent of any 
specific programming-language type. There
+might be multiple Java classes that all have the same schema (for example a 
Protocol-Buffer class or a POJO class),
+and Beam will allow us to seamlessly convert between these types. Schemas also 
provide a simple way to reason about 
+types across different programming-language APIs.
+
+A `PCollection` with a schema does not need to have a `Coder` specified, as 
Beam knows how to encode and decode 
+Schema rows.
+
+### 6.2. Schemas for programming language types {#schemas-for-pl-types}
+While schemas themselves are language independent, they are designed to embed 
naturally into the programming languages
+of the Beam SDK being used. This allows Beam users to continue using native 
types while reaping the advantage of 
+having Beam understand their element schemas.
+ 
+ {:.language-java}
+ In Java you could use the following set of classes to represent the purchase 
schema.  Beam will automatically  
+ infer the correct schema based on the members of the class.
+
+```java
+@DefaultSchema(JavaBeanSchema.class)
+public class Purchase {
+  public String getUserId();  // Returns the id of the user who made the 
purchase.
+  public long getItemId();  // Returns the identifier of the item that was 
purchased.
+  public ShippingAddress getShippingAddress();  // Returns the shipping 
address, a nested type.
+  public long getCostCents();  // Returns the cost of the item.
+  public List<Transaction> getTransactions();  // Returns the transactions 
that paid for this purchase (returns a list, since the purchase might be spread 
out over multiple credit cards).
+  
+  @SchemaCreate
+  public Purchase(String userId, long itemId, ShippingAddress shippingAddress, 
long costCents, 
+                  List<Transaction> transactions) {
+      ...
+  }
+}
+
+@DefaultSchema(JavaBeanSchema.class)
+public class ShippingAddress {
+  public String getStreetAddress();
+  public String getCity();
+  @Nullable public String getState();
+  public String getCountry();
+  public String getPostCode();
+  
+  @SchemaCreate
+  public ShippingAddress(String streetAddress, String city, @Nullable String 
state, String country,
+                         String postCode) {
+     ...
+  }
+}
+
+@DefaultSchema(JavaBeanSchema.class)
+public class Transaction {
+  public String getBank();
+  public double getPurchaseAmount();
+ 
+  @SchemaCreate
+  public Transaction(String bank, double purchaseAmount) {
+     ...
+  }
+}
+```
+
+Using JavaBean classes as above is one way to map a schema to Java classes. 
However multiple Java classes might have
+the same schema, in which case the different Java types can often be used 
interchangeably. For example, the above
+`Transaction` class has the same schema as the following class:
+
+```java
+@DefaultSchema(JavaFieldSchema.class)
+public class TransactionPojo {
+  public String bank;
+  public double purchaseAmount;
+}
+```
+
+So if we had two `PCollection`s as follows
+
+```java
+PCollection<Transaction> transactionBeans = readTransactionsAsJavaBean();
+PCollection<TransactionPojos> transactionPojos = readTransactionsAsPojo();
+```
+
+Then these two `PCollection`s would have the same schema, even though their 
Java types would be different. This means
+for example the the following two code snippets are valid:
+
+```java
+transactionBeans.apply(ParDo.of(new DoFn<...>() {
+   @ProcessElement public void process(@Element TransactionPojo pojo) {
+      ...
+   }
+}));
+```
+
+and
+```java
+transactionPojos.apply(ParDo.of(new DoFn<...>() {
+   @ProcessElement public void process(@Element Transaction row) {
+    }
+}));
+```
+
+Even though the in both cases the `@Element` parameter differs from the the 
`PCollection`'s Java type, since the
+schemas are the same Beam will automatically make the conversion. The built-in 
`Convert` transform can also be used
+to translate between Java types of equivalent schemas, as detailed below.
+
+### 6.3. Schema definition {#schema-definition}
+The schema for a `PCollection` defines elements of that `PCollection` as an 
ordered list of named fields. Each field
+has a name, a type, and possibly a set of user options. The type of a field 
can be primitive or composite. The following
+are the primitive types currently supported by Beam:
+
+<table>
+  <thead>
+    <tr class="header">
+      <th>Type</th>
+      <th>Description</th>
+    </tr>
+  </thead>
+  <tbody>
+    <tr>
+      <td>BYTE</td>
+      <td>An 8-bit signed value</td>
+    </tr>
+     <tr>
+       <td>INT16</td>
+       <td>A 16-bit signed value</td>
+     </tr>
+     <tr>
+       <td>INT32</td>
+       <td>A 32-bit signed value</td>
+     </tr>
+    <tr>
+      <td>INT64</td>
+       <td>A 64-bit signed value</td>
+     </tr>
+     <tr>
+       <td>DECIMAL</td>
+       <td>An arbitrary-precision decimal type</td>
+     </tr>
+     <tr>
+       <td>FLOAT</td>
+       <td>A 32-bit IEEE 754 floating point number</td>
+     </tr>
+     <tr>
+       <td>DOUBLE</td>
+       <td>A 64-bit IEEE 754 floating point number</td>
+     </tr>
+     <tr>
+       <td>STRING</td>
+       <td>A string</td>
+     </tr>
+     <tr>
+       <td>DATETIME</td>
+       <td>A timestamp represented as milliseconds since the epoch</td>
+     </tr>  
+     <tr>
+       <td>BOOLEAN</td>
+       <td>A boolean value</td>
+     </tr>
+     <tr>
+       <td>BYTES</td>
+       <td>A raw byte array</td>
+     </tr>             
+  </tbody>
+</table>
+<br/>
+
+A field can also reference a nested schema. In this case, the field will have 
type ROW, and the nested schema will 
+be an attribute of this field type.
+
+Three collection types are supported as field types: ARRAY, ITERABLE and MAP:
+* **ARRAY** This represents a repeated value type, where the repeated elements 
can have any supported type. Arrays of 
+nested rows are supported, as are arrays of arrays.
+* **ITERABLE** This is very similar to the array type, it represents a 
repeated value, but one in which the full list of 
+items is not known until iterated over. This is intended for the case where an 
iterable might be larger than the 
+available memory, and backed by external storage (for example, this can happen 
with the iterable returned by a 
+`GroupByKey`). The repeated elements can have any supported type.
+* **MAP** This represents an associative map from keys to values. All schema 
types are supported for both keys and values.
+ Values that contain map types cannot be used as keys in any grouping 
operation.
+
+### 6.4. Logical types {#logical-types}
+Users can extend the schema type system to add custom logical types that can 
be used as a field. A logical type is 
+identified by a unique identifier and an argument. A logical type also 
specifies an underlying schema type to be used 
+for storage, along with conversions to and from that type. As an example, a 
logical union can always be represented as 
+a row with nullable fields, where the user ensures that only one of those 
fields is ever set at a time. However this can
+be tedious and complex to manage. The OneOf logical type provides a value 
class that makes it easier to manage the type
+as a union, while still using a row with nullable fields as its underlying 
storage. Each logical type also has a 
+unique identifier, so they can be interpreted by other languages as well. More 
examples of logical types are listed 
+below.
+
+#### 6.4.1. Defining a logical type {#defining-a-logical-type}
+To define a logical type you must specify a Schema type to be used to 
represent the underlying type as well as a unique
+identifier for that type. A logical type imposes additional semantics on top a 
schema type. For example, a logical 
+type to represent nanosecond timestamps is represented as a schema containing 
an INT64 and an INT32 field. This schema
+alone does not say anything about how to interpret this type, however the 
logical type tells you that this represents
+a nanosecond timestamp, with the INT64 field representing seconds and the 
INT32 field representing nanoseconds.
+
+Logical types are also specified by an argument, which allows creating a class 
of related typed. For example, a 
+limited-precision decimal type would have an integer argument indicating how 
many digits of precision are represented.
+The argument is represented by a schema type, so can itself be a complex type.
+
+ {:.language-java}
+In Java, a logical type is specified as a subclass of the `LogicalType` class. 
A custom Java class can be specified to 
+represent the logical type and conversion functions must be supplied to 
convert back and forth between this Java class
+and the underlying Schema type representation. For example, the logical type 
representing nanosecond timestamp might
+be implemented as follows
+
+```java
+// A Logical type using java.time.Instant to represent the logical type.
+public class TimestampNanos implements LogicalType<Instant, Row> {
+  // The underlying schema used to represent rows.
+  private final Schema SCHEMA = 
Schema.builder().addInt64Field("seconds").addInt32Field("nanos").build();
+  @Override public String getIdentifier() { return "timestampNanos"; }
+  @Override public FieldType getBaseType() { return schema; }
+  
+  // Convert the representation type to the underlying Row type. Called by 
Beam when necessary.
+  @Override public Row toBaseType(Instant instant) {
+    return Row.withSchema(schema).addValues(instant.getEpochSecond(), 
instant.getNano()).build();
+  }
+  
+  // Convert the underlying Row type to and Instant. Called by Beam when 
necessary.
+  @Override public Instant toInputType(Row base) {
+    return Instant.of(row.getInt64("seconds"), row.getInt32("nanos"));
+  }
+
+     ...
+}
+```
+
+#### 6.4.2. Useful logical types {#built-in-logical-types}
+##### **EnumerationType**
+EnumerationType allows creating an enumeration type consisting of a set of 
named constants.
 
 Review comment:
   I think the ask is to avoid repeating EnumerationType in the description by 
referring to it as "This logical type"

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to