[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-09 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282458283
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java
 ##
 @@ -0,0 +1,427 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Describes the data type of a value in the table ecosystem. Instances of 
this class can be used to
+ * declare input and/or output types of operations.
+ *
+ * The {@link DataType} class has two responsibilities: declaring a logical 
type and giving hints
+ * about the physical representation of data to the optimizer. While the 
logical type is mandatory,
+ * hints are optional but useful at the edges to other APIs.
+ *
+ * The logical type is independent of any physical representation and is 
close to the "data type"
+ * terminology of the SQL standard. See {@link 
org.apache.flink.table.types.logical.LogicalType} and
+ * its subclasses for more information about available logical types and their 
properties.
+ *
+ * Physical hints are required at the edges of the table ecosystem. Hints 
indicate the data format
+ * that an implementation expects. For example, a data source could express 
that it produces values for
+ * logical timestamps using a {@link java.sql.Timestamp} class instead of 
using {@link java.time.LocalDateTime}.
+ * With this information, the runtime is able to convert the produced class 
into its internal data
+ * format. In return, a data sink can declare the data format it consumes from 
the runtime.
+ *
+ * @see DataTypes for a list of supported data types and instances of this 
class.
+ */
+@PublicEvolving
+public abstract class DataType implements Serializable {
+
+   protected LogicalType logicalType;
+
+   protected @Nullable Class conversionClass;
+
+   private DataType(LogicalType logicalType, @Nullable Class 
conversionClass) {
+   this.logicalType = Preconditions.checkNotNull(logicalType, 
"Logical type must not be null.");
+   this.conversionClass = performEarlyClassValidation(logicalType, 
conversionClass);
+   }
+
+   /**
+* Returns the corresponding logical type.
+*
+* @return a parameterized instance of {@link LogicalType}
+*/
+   public LogicalType getLogicalType() {
+   return logicalType;
+   }
+
+   /**
+* Returns the corresponding conversion class for representing values. 
If no conversion class was
+* defined manually, the default conversion defined by the logical type 
is used.
+*
+* @see LogicalType#getDefaultConversion()
+*
+* @return the expected conversion class
+*/
+   public Class getConversionClass() {
+   if (conversionClass == null) {
+   return logicalType.getDefaultConversion();
+   }
+   return conversionClass;
+   }
+
+   /**
+* Adds a hint that null values are not expected in the data for this 
type.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType notNull();
+
+   /**
+* Adds a hint that null values are expected in the data for this type 
(default behavior).
+*
+* This method exists for explicit declaration of the default 
behavior or for invalidation of
+* a previous call to {@link #notNull()}.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType andNull();
+
+   /**
+* Adds a hint that data should be represented using th

[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-09 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282388964
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java
 ##
 @@ -0,0 +1,427 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Describes the data type of a value in the table ecosystem. Instances of 
this class can be used to
+ * declare input and/or output types of operations.
+ *
+ * The {@link DataType} class has two responsibilities: declaring a logical 
type and giving hints
+ * about the physical representation of data to the optimizer. While the 
logical type is mandatory,
+ * hints are optional but useful at the edges to other APIs.
+ *
+ * The logical type is independent of any physical representation and is 
close to the "data type"
+ * terminology of the SQL standard. See {@link 
org.apache.flink.table.types.logical.LogicalType} and
+ * its subclasses for more information about available logical types and their 
properties.
+ *
+ * Physical hints are required at the edges of the table ecosystem. Hints 
indicate the data format
+ * that an implementation expects. For example, a data source could express 
that it produces values for
+ * logical timestamps using a {@link java.sql.Timestamp} class instead of 
using {@link java.time.LocalDateTime}.
+ * With this information, the runtime is able to convert the produced class 
into its internal data
+ * format. In return, a data sink can declare the data format it consumes from 
the runtime.
+ *
+ * @see DataTypes for a list of supported data types and instances of this 
class.
+ */
+@PublicEvolving
+public abstract class DataType implements Serializable {
+
+   protected LogicalType logicalType;
+
+   protected @Nullable Class conversionClass;
+
+   private DataType(LogicalType logicalType, @Nullable Class 
conversionClass) {
+   this.logicalType = Preconditions.checkNotNull(logicalType, 
"Logical type must not be null.");
+   this.conversionClass = performEarlyClassValidation(logicalType, 
conversionClass);
+   }
+
+   /**
+* Returns the corresponding logical type.
+*
+* @return a parameterized instance of {@link LogicalType}
+*/
+   public LogicalType getLogicalType() {
+   return logicalType;
+   }
+
+   /**
+* Returns the corresponding conversion class for representing values. 
If no conversion class was
+* defined manually, the default conversion defined by the logical type 
is used.
+*
+* @see LogicalType#getDefaultConversion()
+*
+* @return the expected conversion class
+*/
+   public Class getConversionClass() {
+   if (conversionClass == null) {
+   return logicalType.getDefaultConversion();
+   }
+   return conversionClass;
+   }
+
+   /**
+* Adds a hint that null values are not expected in the data for this 
type.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType notNull();
+
+   /**
+* Adds a hint that null values are expected in the data for this type 
(default behavior).
+*
+* This method exists for explicit declaration of the default 
behavior or for invalidation of
+* a previous call to {@link #notNull()}.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType andNull();
+
+   /**
+* Adds a hint that data should be represented using th

[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-09 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282381642
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java
 ##
 @@ -0,0 +1,703 @@
+/*
+ * 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.flink.table.api;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.AnyType;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DayTimeIntervalType;
+import 
org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.MultisetType;
+import org.apache.flink.table.types.logical.NullType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.TypeInformationAnyType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.YearMonthIntervalType;
+import 
org.apache.flink.table.types.logical.YearMonthIntervalType.YearMonthResolution;
+import org.apache.flink.table.types.logical.ZonedTimestampType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A {@link DataType} can be used to declare input and/or output types of 
operations. This class
+ * enumerates all supported data types of the Table & SQL API.
+ */
+@PublicEvolving
+public final class DataTypes {
+
+   // we use SQL-like naming for data types and avoid Java keyword clashes
+   // CHECKSTYLE.OFF: MethodName
+
+   /**
+* Data type of a fixed-length character string {@code CHAR(n)} where 
{@code n} is the number
+* of code points. {@code n} must have a value between 1 and 255 (both 
inclusive).
+*
+* @see CharType
+*/
+   public static DataType.AtomicDataType CHAR(int n) {
 
 Review comment:
   Good point. First I was a bit skeptical because it would require casting in 
util methods but we can also provide a visitor.


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


With regards,
Apache Git Services


[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-09 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282380331
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java
 ##
 @@ -0,0 +1,427 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Describes the data type of a value in the table ecosystem. Instances of 
this class can be used to
+ * declare input and/or output types of operations.
+ *
+ * The {@link DataType} class has two responsibilities: declaring a logical 
type and giving hints
+ * about the physical representation of data to the optimizer. While the 
logical type is mandatory,
+ * hints are optional but useful at the edges to other APIs.
+ *
+ * The logical type is independent of any physical representation and is 
close to the "data type"
+ * terminology of the SQL standard. See {@link 
org.apache.flink.table.types.logical.LogicalType} and
+ * its subclasses for more information about available logical types and their 
properties.
+ *
+ * Physical hints are required at the edges of the table ecosystem. Hints 
indicate the data format
+ * that an implementation expects. For example, a data source could express 
that it produces values for
+ * logical timestamps using a {@link java.sql.Timestamp} class instead of 
using {@link java.time.LocalDateTime}.
+ * With this information, the runtime is able to convert the produced class 
into its internal data
+ * format. In return, a data sink can declare the data format it consumes from 
the runtime.
+ *
+ * @see DataTypes for a list of supported data types and instances of this 
class.
+ */
+@PublicEvolving
+public abstract class DataType implements Serializable {
+
+   protected LogicalType logicalType;
+
+   protected @Nullable Class conversionClass;
+
+   private DataType(LogicalType logicalType, @Nullable Class 
conversionClass) {
+   this.logicalType = Preconditions.checkNotNull(logicalType, 
"Logical type must not be null.");
+   this.conversionClass = performEarlyClassValidation(logicalType, 
conversionClass);
+   }
+
+   /**
+* Returns the corresponding logical type.
+*
+* @return a parameterized instance of {@link LogicalType}
+*/
+   public LogicalType getLogicalType() {
+   return logicalType;
+   }
+
+   /**
+* Returns the corresponding conversion class for representing values. 
If no conversion class was
+* defined manually, the default conversion defined by the logical type 
is used.
+*
+* @see LogicalType#getDefaultConversion()
+*
+* @return the expected conversion class
+*/
+   public Class getConversionClass() {
+   if (conversionClass == null) {
+   return logicalType.getDefaultConversion();
+   }
+   return conversionClass;
+   }
+
+   /**
+* Adds a hint that null values are not expected in the data for this 
type.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType notNull();
+
+   /**
+* Adds a hint that null values are expected in the data for this type 
(default behavior).
+*
+* This method exists for explicit declaration of the default 
behavior or for invalidation of
+* a previous call to {@link #notNull()}.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType andNull();
+
+   /**
+* Adds a hint that data should be represented using th

[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-09 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282361681
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java
 ##
 @@ -0,0 +1,427 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Describes the data type of a value in the table ecosystem. Instances of 
this class can be used to
+ * declare input and/or output types of operations.
+ *
+ * The {@link DataType} class has two responsibilities: declaring a logical 
type and giving hints
+ * about the physical representation of data to the optimizer. While the 
logical type is mandatory,
+ * hints are optional but useful at the edges to other APIs.
+ *
+ * The logical type is independent of any physical representation and is 
close to the "data type"
+ * terminology of the SQL standard. See {@link 
org.apache.flink.table.types.logical.LogicalType} and
+ * its subclasses for more information about available logical types and their 
properties.
+ *
+ * Physical hints are required at the edges of the table ecosystem. Hints 
indicate the data format
+ * that an implementation expects. For example, a data source could express 
that it produces values for
+ * logical timestamps using a {@link java.sql.Timestamp} class instead of 
using {@link java.time.LocalDateTime}.
+ * With this information, the runtime is able to convert the produced class 
into its internal data
+ * format. In return, a data sink can declare the data format it consumes from 
the runtime.
+ *
+ * @see DataTypes for a list of supported data types and instances of this 
class.
+ */
+@PublicEvolving
+public abstract class DataType implements Serializable {
+
+   protected LogicalType logicalType;
+
+   protected @Nullable Class conversionClass;
+
+   private DataType(LogicalType logicalType, @Nullable Class 
conversionClass) {
+   this.logicalType = Preconditions.checkNotNull(logicalType, 
"Logical type must not be null.");
+   this.conversionClass = performEarlyClassValidation(logicalType, 
conversionClass);
+   }
+
+   /**
+* Returns the corresponding logical type.
+*
+* @return a parameterized instance of {@link LogicalType}
+*/
+   public LogicalType getLogicalType() {
+   return logicalType;
+   }
+
+   /**
+* Returns the corresponding conversion class for representing values. 
If no conversion class was
+* defined manually, the default conversion defined by the logical type 
is used.
+*
+* @see LogicalType#getDefaultConversion()
+*
+* @return the expected conversion class
+*/
+   public Class getConversionClass() {
+   if (conversionClass == null) {
+   return logicalType.getDefaultConversion();
+   }
+   return conversionClass;
+   }
+
+   /**
+* Adds a hint that null values are not expected in the data for this 
type.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType notNull();
+
+   /**
+* Adds a hint that null values are expected in the data for this type 
(default behavior).
+*
+* This method exists for explicit declaration of the default 
behavior or for invalidation of
+* a previous call to {@link #notNull()}.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType andNull();
 
 Review comment:
   Yes, I'm also ok with `notNull()` and `nullable()`. Any

[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-09 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282361226
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java
 ##
 @@ -0,0 +1,427 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Describes the data type of a value in the table ecosystem. Instances of 
this class can be used to
+ * declare input and/or output types of operations.
+ *
+ * The {@link DataType} class has two responsibilities: declaring a logical 
type and giving hints
+ * about the physical representation of data to the optimizer. While the 
logical type is mandatory,
+ * hints are optional but useful at the edges to other APIs.
+ *
+ * The logical type is independent of any physical representation and is 
close to the "data type"
+ * terminology of the SQL standard. See {@link 
org.apache.flink.table.types.logical.LogicalType} and
+ * its subclasses for more information about available logical types and their 
properties.
+ *
+ * Physical hints are required at the edges of the table ecosystem. Hints 
indicate the data format
+ * that an implementation expects. For example, a data source could express 
that it produces values for
+ * logical timestamps using a {@link java.sql.Timestamp} class instead of 
using {@link java.time.LocalDateTime}.
+ * With this information, the runtime is able to convert the produced class 
into its internal data
+ * format. In return, a data sink can declare the data format it consumes from 
the runtime.
+ *
+ * @see DataTypes for a list of supported data types and instances of this 
class.
+ */
+@PublicEvolving
+public abstract class DataType implements Serializable {
+
+   protected LogicalType logicalType;
+
+   protected @Nullable Class conversionClass;
+
+   private DataType(LogicalType logicalType, @Nullable Class 
conversionClass) {
+   this.logicalType = Preconditions.checkNotNull(logicalType, 
"Logical type must not be null.");
+   this.conversionClass = performEarlyClassValidation(logicalType, 
conversionClass);
+   }
+
+   /**
+* Returns the corresponding logical type.
+*
+* @return a parameterized instance of {@link LogicalType}
+*/
+   public LogicalType getLogicalType() {
+   return logicalType;
+   }
+
+   /**
+* Returns the corresponding conversion class for representing values. 
If no conversion class was
+* defined manually, the default conversion defined by the logical type 
is used.
+*
+* @see LogicalType#getDefaultConversion()
+*
+* @return the expected conversion class
+*/
+   public Class getConversionClass() {
+   if (conversionClass == null) {
+   return logicalType.getDefaultConversion();
+   }
+   return conversionClass;
+   }
+
+   /**
+* Adds a hint that null values are not expected in the data for this 
type.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType notNull();
+
+   /**
+* Adds a hint that null values are expected in the data for this type 
(default behavior).
+*
+* This method exists for explicit declaration of the default 
behavior or for invalidation of
+* a previous call to {@link #notNull()}.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType andNull();
+
+   /**
+* Adds a hint that data should be represented using th

[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-08 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282348005
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/DataTypes.java
 ##
 @@ -0,0 +1,703 @@
+/*
+ * 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.flink.table.api;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.AnyType;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.BinaryType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.CharType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DayTimeIntervalType;
+import 
org.apache.flink.table.types.logical.DayTimeIntervalType.DayTimeResolution;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LocalZonedTimestampType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.MultisetType;
+import org.apache.flink.table.types.logical.NullType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimeType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.table.types.logical.TypeInformationAnyType;
+import org.apache.flink.table.types.logical.VarBinaryType;
+import org.apache.flink.table.types.logical.VarCharType;
+import org.apache.flink.table.types.logical.YearMonthIntervalType;
+import 
org.apache.flink.table.types.logical.YearMonthIntervalType.YearMonthResolution;
+import org.apache.flink.table.types.logical.ZonedTimestampType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * A {@link DataType} can be used to declare input and/or output types of 
operations. This class
+ * enumerates all supported data types of the Table & SQL API.
+ */
+@PublicEvolving
+public final class DataTypes {
+
+   // we use SQL-like naming for data types and avoid Java keyword clashes
+   // CHECKSTYLE.OFF: MethodName
+
+   /**
+* Data type of a fixed-length character string {@code CHAR(n)} where 
{@code n} is the number
+* of code points. {@code n} must have a value between 1 and 255 (both 
inclusive).
+*
+* @see CharType
+*/
+   public static DataType.AtomicDataType CHAR(int n) {
+   return new DataType.AtomicDataType(new CharType(n));
+   }
+
+   /**
+* Data type of a variable-length character string {@code VARCHAR(n)} 
where {@code n} is the
+* maximum number of code points. {@code n} must have a value between 1 
and {@link Integer#MAX_VALUE}
+* (both inclusive).
+*
+* @see VarCharType
+*/
+   public static DataType.AtomicDataType VARCHAR(int n) {
+   return new DataType.AtomicDataType(new VarCharType(n));
+   }
+
+   /**
+* Data type of a variable-length character string with defined maximum 
length. This is a shortcut
+* for {@code VARCHAR(2147483647)} for representing JVM strings.
+*
+* @see VarCharType
+  

[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-08 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282347857
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java
 ##
 @@ -0,0 +1,427 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Describes the data type of a value in the table ecosystem. Instances of 
this class can be used to
+ * declare input and/or output types of operations.
+ *
+ * The {@link DataType} class has two responsibilities: declaring a logical 
type and giving hints
+ * about the physical representation of data to the optimizer. While the 
logical type is mandatory,
+ * hints are optional but useful at the edges to other APIs.
+ *
+ * The logical type is independent of any physical representation and is 
close to the "data type"
+ * terminology of the SQL standard. See {@link 
org.apache.flink.table.types.logical.LogicalType} and
+ * its subclasses for more information about available logical types and their 
properties.
+ *
+ * Physical hints are required at the edges of the table ecosystem. Hints 
indicate the data format
+ * that an implementation expects. For example, a data source could express 
that it produces values for
+ * logical timestamps using a {@link java.sql.Timestamp} class instead of 
using {@link java.time.LocalDateTime}.
+ * With this information, the runtime is able to convert the produced class 
into its internal data
+ * format. In return, a data sink can declare the data format it consumes from 
the runtime.
+ *
+ * @see DataTypes for a list of supported data types and instances of this 
class.
+ */
+@PublicEvolving
+public abstract class DataType implements Serializable {
+
+   protected LogicalType logicalType;
+
+   protected @Nullable Class conversionClass;
+
+   private DataType(LogicalType logicalType, @Nullable Class 
conversionClass) {
+   this.logicalType = Preconditions.checkNotNull(logicalType, 
"Logical type must not be null.");
+   this.conversionClass = performEarlyClassValidation(logicalType, 
conversionClass);
+   }
+
+   /**
+* Returns the corresponding logical type.
+*
+* @return a parameterized instance of {@link LogicalType}
+*/
+   public LogicalType getLogicalType() {
+   return logicalType;
+   }
+
+   /**
+* Returns the corresponding conversion class for representing values. 
If no conversion class was
+* defined manually, the default conversion defined by the logical type 
is used.
+*
+* @see LogicalType#getDefaultConversion()
+*
+* @return the expected conversion class
+*/
+   public Class getConversionClass() {
+   if (conversionClass == null) {
+   return logicalType.getDefaultConversion();
+   }
+   return conversionClass;
+   }
+
+   /**
+* Adds a hint that null values are not expected in the data for this 
type.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType notNull();
+
+   /**
+* Adds a hint that null values are expected in the data for this type 
(default behavior).
+*
+* This method exists for explicit declaration of the default 
behavior or for invalidation of
+* a previous call to {@link #notNull()}.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType andNull();
 
 Review comment:
   1. The goal was to be close to SQL: `INT NOT NULL` -> `

[GitHub] [flink] twalthr commented on a change in pull request #8360: [FLINK-12393][table-common] Add the user-facing classes of the new type system

2019-05-08 Thread GitBox
twalthr commented on a change in pull request #8360: 
[FLINK-12393][table-common] Add the user-facing classes of the new type system
URL: https://github.com/apache/flink/pull/8360#discussion_r282347087
 
 

 ##
 File path: 
flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/DataType.java
 ##
 @@ -0,0 +1,427 @@
+/*
+ * 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.flink.table.types;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Describes the data type of a value in the table ecosystem. Instances of 
this class can be used to
+ * declare input and/or output types of operations.
+ *
+ * The {@link DataType} class has two responsibilities: declaring a logical 
type and giving hints
+ * about the physical representation of data to the optimizer. While the 
logical type is mandatory,
+ * hints are optional but useful at the edges to other APIs.
+ *
+ * The logical type is independent of any physical representation and is 
close to the "data type"
+ * terminology of the SQL standard. See {@link 
org.apache.flink.table.types.logical.LogicalType} and
+ * its subclasses for more information about available logical types and their 
properties.
+ *
+ * Physical hints are required at the edges of the table ecosystem. Hints 
indicate the data format
+ * that an implementation expects. For example, a data source could express 
that it produces values for
+ * logical timestamps using a {@link java.sql.Timestamp} class instead of 
using {@link java.time.LocalDateTime}.
+ * With this information, the runtime is able to convert the produced class 
into its internal data
+ * format. In return, a data sink can declare the data format it consumes from 
the runtime.
+ *
+ * @see DataTypes for a list of supported data types and instances of this 
class.
+ */
+@PublicEvolving
+public abstract class DataType implements Serializable {
+
+   protected LogicalType logicalType;
+
+   protected @Nullable Class conversionClass;
+
+   private DataType(LogicalType logicalType, @Nullable Class 
conversionClass) {
+   this.logicalType = Preconditions.checkNotNull(logicalType, 
"Logical type must not be null.");
+   this.conversionClass = performEarlyClassValidation(logicalType, 
conversionClass);
+   }
+
+   /**
+* Returns the corresponding logical type.
+*
+* @return a parameterized instance of {@link LogicalType}
+*/
+   public LogicalType getLogicalType() {
+   return logicalType;
+   }
+
+   /**
+* Returns the corresponding conversion class for representing values. 
If no conversion class was
+* defined manually, the default conversion defined by the logical type 
is used.
+*
+* @see LogicalType#getDefaultConversion()
+*
+* @return the expected conversion class
+*/
+   public Class getConversionClass() {
+   if (conversionClass == null) {
+   return logicalType.getDefaultConversion();
+   }
+   return conversionClass;
+   }
+
+   /**
+* Adds a hint that null values are not expected in the data for this 
type.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType notNull();
+
+   /**
+* Adds a hint that null values are expected in the data for this type 
(default behavior).
+*
+* This method exists for explicit declaration of the default 
behavior or for invalidation of
+* a previous call to {@link #notNull()}.
+*
+* @return a new, reconfigured data type instance
+*/
+   public abstract DataType andNull();
+
+   /**
+* Adds a hint that data should be represented using th